Apache Airflow 2 vs 3: A Deep Technical Comparison for Data Engineers ๐
TL;DR โ Airflow 3 dissolves the monolithic webserver into three independent
services, strips direct database access from task code, ships a fully stable
Task SDK, and rewrites the entire UI in React. If you are running Airflow 2 in
production, this article will tell you exactly what breaks, what improves, and
how to migrate without losing a night's sleep. ๐ด
Why This Comparison Matters โ๏ธ
Every major Airflow release has nudged the architecture forward. Airflow 2 gave us
the TaskFlow API, the Scheduler high-availability refactor, and provider packages.
Airflow 3 is different in kind, not just degree.
In the process of migrating a production Docker Compose stack for a healthcare ML
retraining pipeline from Airflow 2 patterns to Airflow 3, every single one of the
following hit in production:
- CPU spike to 600% caused by a silent breaking change in JWT key management ๐
- Tasks silently failing with
Connection refusedbecauselocalhostno longer means what it used to ๐ - A healthcheck that always reported unhealthy because port 8974 no longer exists โ
- A user creation step that silently did nothing because FAB is gone ๐ค
Each of these failures traces back to a deliberate, principled architectural
decision in Airflow 3. Once you understand why the changes were made, the fixes
are obvious โ but without that context, Airflow 3 can feel like it is actively
working against you.
This article is that context. ๐ก
The 30-Second Summary โฑ๏ธ
| Dimension | Airflow 2 | Airflow 3 |
|---|---|---|
| UI framework | Flask-AppBuilder (FAB) | React (FastAPI backend) |
| Webserver | airflow webserver |
airflow api-server |
| DAG Processor | Embedded in scheduler | Mandatory separate service |
| Task Execution | Direct fork/subprocess | Task Execution API (AIP-72) |
| Metadata DB access from tasks | Allowed | Prohibited |
| Auth manager default | FAB (full RBAC) | SimpleAuthManager |
| REST API | v1 (Flask) | v2 (FastAPI, stable) |
| Default schedule |
@daily (cron) |
None |
catchup default |
True |
False |
| SequentialExecutor | Available | Removed |
| SubDAGs | Available | Removed |
| SLAs | Available | Removed |
Import path for @dag/@task |
airflow.decorators |
airflow.sdk |
| XCom pickling | Enabled by default | Disabled by default |
| Python minimum | 3.8 | 3.9 |
| PostgreSQL minimum | 12 | 13 |
๐๏ธ Part 1 โ The Architectural Paradigm Shift
Airflow 2: One Webserver to Rule Them All ๐๏ธ
In Airflow 2, the mental model for a self-hosted deployment is relatively
straightforward. You run four processes:
airflow webserver # Flask-AppBuilder UI + REST API v1 + auth
airflow scheduler # parses DAGs + triggers task instances
airflow worker # (CeleryExecutor) executes tasks
postgres/mysql # metadata database
The webserver does double duty โ it serves the browser UI and exposes the REST
API and handles authentication, all from a single Flask application. The
scheduler parses your dags/ directory inline, as part of its own main loop.
This is simple to reason about. It is also a single point of failure for three
completely separate concerns.๐๏ธ
Airflow 3: Separation of Concerns as a First-Class Constraint
Airflow 3 decomposes the monolith into discrete, independently scalable services:๐งฉ
airflow api-server # FastAPI: UI + REST API v2 + auth (replaces webserver)
airflow scheduler # triggers task instances only; NO DAG parsing
airflow dag-processor # mandatory: parses DAGs, writes to serialized_dag table
airflow triggerer # manages deferrable operators
postgres/mysql # metadata database
The key insight: the scheduler in Airflow 3 does not parse DAGs. It reads the
serialized_dag table, which is populated exclusively by the dag-processor service.
If you start a scheduler without a dag-processor, it will start cleanly โ and then
do nothing, because it has no serialized DAGs to schedule.๐๏ธ
# Airflow 2: single scheduler did everything
[Scheduler process]
โโโ Parses dags/ directory
โโโ Updates serialized_dag table
โโโ Checks heartbeats
โโโ Triggers TaskInstances
# Airflow 3: responsibilities split
[dag-processor] [scheduler]
โโโ Parses dags/ โโโ Reads serialized_dag
Updates serialized_dag โโโ Checks heartbeats
โโโ Triggers TaskInstances via Execution API
This split unlocks horizontal scalability. The dag-processor can be scaled
independently on compute-heavy deployments with thousands of DAG files, without
touching the scheduler's scheduling loop latency.โก
Part 2 โ The Task Execution API (AIP-72): The Biggest Change You Haven't Heard Of ๐คซ
How Airflow 2 Ran Tasks
In Airflow 2 with LocalExecutor, task execution worked like this:
- Scheduler identifies a TaskInstance ready to run
- Scheduler forks a subprocess
- Subprocess imports your DAG file directly
- Subprocess calls
task.execute(context) - Task code has unrestricted access to
settings.Session,DagRun,TaskInstancemodels โ the entire Airflow metadata database ๐๏ธ
Step 5 is a footgun. Task code could accidentally (or intentionally) query, modify,
or drop metadata. It tightly coupled your business logic to Airflow internals.๐ฃ
How Airflow 3 Runs Tasks
Airflow 3 introduces a Task Execution API โ a lightweight HTTP interface that
sits between the task subprocess and the metadata database:๐ก๏ธ
[Scheduler] โโtriggersโโโบ [Task Subprocess]
โ
โ HTTP (JWT-authenticated)
โผ
[API Server /execution/]
โ
โผ
[Metadata Database]
Task code no longer talks to the database. It talks to the Execution API, which
enforces a controlled, auditable surface for every metadata operation. Direct
imports like from airflow.models import DagRun inside task code will raise errors
in Airflow 3.๐ซ
The JWT Problem (and Why It Caused a 600% CPU Spike)๐ฅ
The Execution API authenticates requests with JWT tokens. The scheduler signs each
task's token; the api-server verifies it. Both must use the same secret key.
In Airflow 3, if AIRFLOW__API_AUTH__JWT_SECRET is not explicitly set, each service
calls get_signing_key() and generates a random in-memory key. The scheduler's
random key โ the api-server's random key. Every task fails immediately with:
Invalid auth token: Signature verification failed
The fix is one environment variable, shared across all containers:๐ ๏ธ
# docker-compose.yml โ x-airflow-common environment block
AIRFLOW__API_AUTH__JWT_SECRET: "your-static-secret-change-in-prod"
The 600% CPU spike came from a related issue: the api-server, when launched with
--workers > 1 (uvicorn default), spawns worker processes via
multiprocessing.spawn. Each spawned process re-initialises its own random JWT key
and immediately crashes when it receives a token signed by the master process. The
crash loop runs at full speed:๐๏ธ
[api-server] Waiting for child process [12]...
[api-server] Child process [12] died unexpectedly
[api-server] Waiting for child process [13]...
[api-server] Child process [13] died unexpectedly
Fix: enforce a single worker until this is resolved upstream.
command: api-server --workers 1
The EXECUTION_API_SERVER_URL Problem๐
Every scheduler container needs to know where the Execution API lives. The default
is http://localhost:8080/execution/. In a Docker Compose deployment, localhost
inside the scheduler container is the scheduler container's own loopback interface.
The api-server is a different container on a different network namespace.๐
# Airflow 2: localhost was fine (single process model)
# Airflow 3 Docker: localhost = wrong container
Result: every task fails with httpx.ConnectError: [Errno 111] Connection refused,
even when the api-server is perfectly healthy.๐
Fix:
AIRFLOW__CORE__EXECUTION_API_SERVER_URL: "http://airflow-api-server:8080/execution/"
Part 3 โ Authentication: FAB Out, SimpleAuthManager In๐
Flask-AppBuilder in Airflow 2
Airflow 2 used Flask-AppBuilder (FAB) for authentication. FAB gave you:
- Full RBAC with built-in roles (Admin, Op, User, Viewer, Public)
- OAuth integrations (Google, GitHub, LDAP, etc.)
- A complete user management UI
-
_AIRFLOW_WWW_USER_CREATEenvironment variable for bootstrapping admin users๐ ๏ธ
# Airflow 2: works as expected
_AIRFLOW_WWW_USER_CREATE: "true"
_AIRFLOW_WWW_USER_USERNAME: "admin"
_AIRFLOW_WWW_USER_PASSWORD: "admin"
_AIRFLOW_WWW_USER_ROLE: "Admin"
SimpleAuthManager in Airflow 3
Airflow 3 ships SimpleAuthManager as the default. It stores users and passwords in a plain-text JSON file:๐
{
"admin": "my_secure_password"
}
FAB is not gone โ it is available as an explicit provider โ but it is no longer the default. The _AIRFLOW_WWW_USER_CREATE variable is silently ignored when SimpleAuthManager is active. You will see this in your init logs:๐
Skipping user creation as auth manager different from Fab is used
There is no warning that your carefully configured user variables did nothing.โ ๏ธ
To bootstrap a user with SimpleAuthManager in Docker Compose:
# Step 1: configure the users list and passwords file location
AIRFLOW__CORE__SIMPLE_AUTH_MANAGER_USERS: "admin:Admin"
AIRFLOW__CORE__SIMPLE_AUTH_MANAGER_PASSWORDS_FILE: "/opt/airflow/project/simple_auth_manager_passwords.json"
# Step 2: write the passwords file in your init container
command:
- -c
- |
python3 -c "
import json
open('/opt/airflow/project/simple_auth_manager_passwords.json','w').write(
json.dumps({'admin': 'your_password'})
)"
exec /entrypoint airflow version
The passwords file must be accessible to all containers โ use a shared bind mount.๐
Choosing Between SimpleAuthManager and FAB
| Scenario | Recommendation |
|---|---|
| Local dev / CI / demos | SimpleAuthManager โ fast, zero config |
| Small team, basic username/password | SimpleAuthManager |
| Enterprise SSO (LDAP, OAuth, SAML) | FAB provider (apache-airflow-providers-fab) |
| Multi-team RBAC with fine-grained permissions | FAB provider |
| Kubernetes deployments | FAB provider or custom AuthManager implementation |
Part 4 โ Breaking Changes Catalogue๐
4.1 SubDAGs โ TaskGroups and Assets๐ฆ
SubDAGs are removed in Airflow 3. They were always problematic โ they introduced deadlock risks with pool management, made the graph view confusing, and performed poorly at scale.๐
# Airflow 2 (SubDAG pattern โ do not migrate this verbatim)
from airflow.operators.subdag import SubDagOperator
process_data = SubDagOperator(
task_id="process_data",
subdag=create_subdag(dag.dag_id, "process_data", args),
dag=dag
)
# Airflow 3 migration: TaskGroups for visual grouping
from airflow.sdk import dag, task
from airflow.utils.task_group import TaskGroup
@dag(schedule=None)
def my_pipeline():
with TaskGroup("process_data") as process_data:
@task
def validate(): ...
@task
def transform(): ...
validate() >> transform()
For cross-DAG dependencies that SubDAGs were sometimes used for, the preferred Airflow 3 pattern is Asset-based scheduling:๐
from airflow.sdk import Asset
raw_data = Asset("s3://my-bucket/raw/")
@dag(schedule=[raw_data]) # this DAG runs when raw_data is updated
def downstream_pipeline(): ...
4.2 SequentialExecutor Removed๐ซ
SequentialExecutor (runs one task at a time, no parallelism) is gone. The replacement for local development is LocalExecutor with a PostgreSQL or SQLite backend.๐๏ธ
# Airflow 2: SequentialExecutor was the default for fresh installs
AIRFLOW__CORE__EXECUTOR: SequentialExecutor
# Airflow 3: use LocalExecutor
AIRFLOW__CORE__EXECUTOR: LocalExecutor
Note: LocalExecutor requires a real database backend (PostgreSQL recommended). SQLite with LocalExecutor is technically functional but unsupported for production.โ ๏ธ
4.3 SLA Misses Removedโฐ
The SLA miss feature is gone. It was notoriously unreliable โ callbacks fired inconsistently depending on scheduler restart timing, and the implementation was tightly coupled to the old execution model.๐๏ธ
# Airflow 2 (no longer works in Airflow 3)
@dag(
sla_miss_callback=my_sla_handler
)
def my_dag():
slow_task = PythonOperator(
task_id="slow_task",
python_callable=run_slow_thing,
sla=timedelta(hours=2) # removed
)
Migration options:๐ ๏ธ
- Airflow 3.2+: Use Deadline Alerts (scheduler-native, much more reliable)
- External monitoring: Instrument task duration in your observability stack (Prometheus, Datadog, etc.) and alert from there
4.4 REST API v1 Removed โ FastAPI v2๐
The REST API v1 (Flask-based, under /api/v1/) is completely removed. Airflow 3 ships a stable, FastAPI-backed REST API under /api/v2/.๐
The v2 API is not backward-compatible. Common breakage points:๐งจ
# v1 endpoint (broken in Airflow 3)
GET /api/v1/dags/{dag_id}/dagRuns
# v2 endpoint (Airflow 3)
GET /api/v2/dags/{dag_id}/dagRuns
Beyond the URL prefix change, the response schemas have also changed. Any custom integrations, CI scripts, or tooling that hit the Airflow API directly will require updates.๐ ๏ธ
The new health endpoint is:๐ฉบ
GET /api/v2/monitor/health
{
"metadatabase": {"status": "healthy"},
"scheduler": {"status": "healthy"},
"triggerer": {"status": "healthy"},
"dag_processor": {"status": "healthy"}
}
Note that dag_processor is a new key โ it did not exist in Airflow 2 health responses.๐
4.5 Removed Context Variables๐ท๏ธ
Several context variables that were available in TaskInstance.context are removed:
# These no longer exist in Airflow 3 task context
execution_date # use logical_date
tomorrow_ds # compute manually
yesterday_ds # compute manually
prev_ds # compute manually
prev_execution_date # removed
next_execution_date # removed
The execution_date rename to logical_date reflects a deeper semantic change: in Airflow 3, logical_date represents run_after (when the DAG should run) rather than data_interval_start (the start of the data window). For event-driven and manual DAGs, this distinction matters.๐ง
# Airflow 2
def my_task(**context):
run_date = context["execution_date"] # deprecated
# Airflow 3
def my_task(**context):
run_date = context["logical_date"] # correct
4.6 XCom Pickling Disabled๐ฅ
XCom pickling is disabled by default in Airflow 3. In Airflow 2, Python objects were serialized via pickle and stored in the metadata database. This allowed arbitrary Python objects to flow between tasks but introduced security risks (arbitrary code execution on deserialization) and size limits.๐ก๏ธ
# Airflow 2: this worked silently
@task
def extract():
return {"data": some_sklearn_model} # pickled into XCom
# Airflow 3: raises an error with default XCom backend
# Use JSON-serializable return values or a custom XCom backend
@task
def extract():
return {"rows": 1000, "path": "s3://bucket/output.parquet"} # safe
For large artifacts (models, DataFrames), the recommended pattern is to write to external storage (S3, GCS, local filesystem) and pass only the path as XCom.๐พ
Part 5 โ What's New in Airflow 3โจ
5.1 The airflow.sdk Namespace๐๏ธ
Airflow 3 ships a stable, versioned Task SDK. All DAG authoring primitives now live under airflow.sdk:๐ฆ
# Airflow 2 import paths (still work in early Airflow 3, will be removed)
from airflow.decorators import dag, task
from airflow.models.dag import DAG
from airflow.sensors.base import BaseSensorOperator
from airflow.datasets import Dataset
# Airflow 3 canonical imports
from airflow.sdk import dag, task, DAG, Asset
from airflow.sdk import BaseSensorOperator
The SDK is designed to have a stable interface across minor versions. The intent is that DAGs written against airflow.sdk should be forward-compatible with future Airflow releases without import-path churn.๐
Important for Docker deployments: The airflow.sdk import chain triggers a connection attempt to the Task Execution API at import time. If the api-server is unavailable or CPU-starved, the dag-processor will hang on this import and eventually be SIGKILL'd by its own parse timeout. Fix the api-server first; everything else follows.๐จ
5.2 DAG Versioning (AIP-66)๐
Airflow 3 introduces first-class DAG versioning. Multiple versions of the same DAG can exist simultaneously in the serialized_dag table, and running DagRuns execute against the DAG version they were triggered with โ not the latest version.๐ฐ๏ธ
dag_id: "healthcare_retrain"
โโโ version 1: train โ validate (runs triggered before 2026-04-10)
โโโ version 2: load_data โ train โ validate (runs triggered after 2026-04-10)
This solves a long-standing pain point: in Airflow 2, modifying a DAG while runs were in-flight could corrupt active DagRuns if the task structure changed.โ
5.3 Asset-Based Scheduling (AIP-74, AIP-75)๐
The Airflow 2 Dataset concept has been renamed to Asset and significantly expanded. Assets replace cron-based scheduling for data-driven pipelines:๐
from airflow.sdk import dag, task, Asset
# Producer DAG
raw_asset = Asset("s3://my-datalake/raw/events.parquet")
@dag(schedule="@hourly")
def ingest_events():
@task(outlets=[raw_asset])
def fetch_and_write():
# ... write to S3
pass
fetch_and_write()
# Consumer DAG โ runs when raw_asset is updated, not on a clock
@dag(schedule=[raw_asset])
def process_events():
@task
def transform():
pass
transform()
Assets enable a push-driven scheduling model where downstream DAGs run when their data dependencies are satisfied, not when a clock fires.๐
5.4 Edge Executor (AIP-69)๐
The Edge Executor allows Airflow tasks to run on lightweight remote workers without CeleryExecutor's operational overhead. Workers register with the api-server via HTTP polling and execute tasks locally, making it viable for:๐ฆพ
- IoT / edge compute deployments
- Low-resource VMs that can not run a Celery broker
- Multi-cloud task distribution without VPN tunnels
# airflow.cfg / env var
AIRFLOW__CORE__EXECUTOR: EdgeExecutor
5.5 Scheduler-Managed Backfills (AIP-78)๐
Backfills in Airflow 2 were CLI-driven one-shot operations. Airflow 3 makes backfills first-class scheduler concepts:๐๏ธ
# Airflow 3: create a scheduler-managed backfill
airflow dags backfill create --dag-id my_dag \
--from-date 2024-01-01 --to-date 2024-12-31
# Inspect backfill state
airflow dags backfill list --dag-id my_dag
Scheduler-managed backfills respect pool limits, run in parallel with live DagRuns, and are visible in the UI โ eliminating the "backfill is a black box" experience from Airflow 2.๐ค
5.6 React UI (AIP-38, AIP-84)๐จ
The Airflow 3 UI is a full rewrite in React, backed by the FastAPI REST API v2. Practical implications:๐ฑ๏ธ
- Significantly faster rendering for DAGs with hundreds of tasksโก
- Grid view replaces the old Tree view as the primary timeline view๐
- The legacy Graph view (force-directed) is replaced with a cleaner task-level dependency graph๐
- The UI now works correctly in all modern browsers without Flask session issues๐
- Dark mode is available natively๐
Part 6 โ Import Path Migration Guide๐บ๏ธ
This is the table you want bookmarked during a migration:
| Airflow 2 import | Airflow 3 import |
|---|---|
from airflow.decorators import dag, task |
from airflow.sdk import dag, task |
from airflow.models.dag import DAG |
from airflow.sdk import DAG |
from airflow.sensors.base import BaseSensorOperator |
from airflow.sdk import BaseSensorOperator |
from airflow.datasets import Dataset |
from airflow.sdk import Asset |
from airflow.models import Variable |
from airflow.sdk import Variable |
from airflow.models import Connection |
from airflow.sdk import Connection |
from airflow.operators.python import PythonOperator |
apache-airflow-providers-standard package |
from airflow.operators.bash import BashOperator |
apache-airflow-providers-standard package |
from airflow.sensors.filesystem import FileSensor |
apache-airflow-providers-standard package |
Many common operators (Python, Bash, File sensors) have moved to apache-airflow-providers-standard. Install this package explicitly:๐ ๏ธ
pip install apache-airflow-providers-standard
Automated Migration with Ruff๐ถ
Airflow 3 ships with Ruff lint rules specifically for migration:
pip install "ruff>=0.13.1"
# Check for mandatory breaking changes (AIR301)
ruff check dags/ --select AIR301 --preview
# Auto-fix safe renames
ruff check dags/ --select AIR301 --fix --unsafe-fixes --preview
# Check for recommended updates (AIR302: deprecated-but-not-yet-removed)
ruff check dags/ --select AIR302 --preview
Example output:๐
dags/retrain_dag.py:3:1: AIR301 airflow.decorators.dag is removed in Airflow 3.0. Use airflow.sdk.dag instead.
[*] AIR301 auto-fix available
Part 7 โ Docker Compose: What Breaks, What to Add๐ณ
If you are running Airflow 2 via Docker Compose, here is a precise list of changes required for Airflow 3.
Services to Addโ
airflow-dag-processor:
<<: *airflow-common
command: dag-processor
healthcheck:
test: ["CMD", "airflow", "jobs", "check", "--job-type", "DagProcessorJob", "--local"]
interval: 60s
timeout: 60s
retries: 3
start_period: 300s
restart: always
depends_on:
airflow-init:
condition: service_completed_successfully
Services to Renameโ๏ธ
# Airflow 2
airflow-webserver:
command: webserver
ports:
- "8080:8080"
# Airflow 3
airflow-api-server:
command: api-server --workers 1 # --workers 1 is critical (see Part 2)
ports:
- "8080:8080"
Environment Variables to Add๐
x-airflow-common:
&airflow-common
environment:
# Critical: prevents Connection refused in scheduler
AIRFLOW__CORE__EXECUTION_API_SERVER_URL: "http://airflow-api-server:8080/execution/"
# Critical: prevents JWT Signature verification failed
AIRFLOW__API_AUTH__JWT_SECRET: "change-this-in-production"
# Required for SimpleAuthManager user configuration
AIRFLOW__CORE__SIMPLE_AUTH_MANAGER_USERS: "admin:Admin"
AIRFLOW__CORE__SIMPLE_AUTH_MANAGER_PASSWORDS_FILE: "/opt/airflow/project/simple_auth_manager_passwords.json"
Healthcheck Changes๐ฉบ
# Airflow 2 scheduler healthcheck (port 8974 no longer exists in Airflow 3)
healthcheck:
test: ["CMD", "curl", "--fail", "http://localhost:8974/health"]
# Airflow 3 scheduler healthcheck
healthcheck:
test: ["CMD", "airflow", "jobs", "check", "--job-type", "SchedulerJob", "--local"]
interval: 60s
timeout: 60s # airflow jobs check takes ~42s (full Python + DB round-trip)
retries: 3
start_period: 300s # covers pip install time on first start
The --local flag is essential. --hostname $(hostname) compares the container's $HOSTNAME env var against the hostname Airflow registered in the database โ these often differ (9811c4ea8dec vs airflow-scheduler.internal), causing perpetual unhealthy status even when the service is running correctly.๐
Part 8 โ Configuration Migrationโ๏ธ
Changed Defaults That Will Surprise You๐ฎ
catchup_by_default: was True in Airflow 2, False in Airflow 3
# If you have DAGs with start_date in the past and no explicit catchup=True,
# they will NOT backfill on first deploy โ this is usually what you want,
# but verify before deploying
[scheduler]
catchup_by_default = False # Airflow 3 default
# Default schedule: was @daily implicit in some contexts, now None
# DAGs with no schedule parameter will not run automatically
[scheduler]
# Use schedule=None explicitly if that's your intent
Renamed Configuration Keysโ๏ธ
# Airflow 2 โ Airflow 3 config key mapping
[webserver]
web_server_host = 0.0.0.0 โ [api]
host = 0.0.0.0
[webserver]
error_logfile = ... โ REMOVED (no replacement)
Automated Config Migration๐ ๏ธ
# Check your airflow.cfg for deprecated/invalid keys
airflow config lint
# Apply automatic fixes
airflow config update --fix
Part 9 โ Migration Path๐ค๏ธ
If you are upgrading a production Airflow 2 deployment, follow this sequence:
Phase 1 โ Prepare (Still on Airflow 2)๐๏ธ
-
Upgrade to Airflow 2.7+ โ the schema migration from earlier versions significantly increases
airflow db migratetime; get that done first.โณ -
Clean the metadata database โ
airflow db cleanremoves old DagRun/TaskInstance records and dramatically speeds up the schema migration.๐งน -
Run Ruff AIR301 checks โ
ruff check dags/ --select AIR301 --preview.๐ถ - Fix all deprecation warnings โ zero warnings in Airflow 2.9 means fewer surprises in Airflow 3.โ ๏ธ
-
Audit direct database access โ grep your task code for
from airflow.modelsimports; these will break.๐
# Find tasks using direct metadata DB access
grep -r "from airflow.models" dags/ --include="*.py"
grep -r "settings.Session" dags/ --include="*.py"
grep -r "DagRun|TaskInstance|Variable" dags/ --include="*.py" | grep "import"
Phase 2 โ Upgradeโฌ๏ธ
- Back up your metadata database โ non-negotiable.๐พ
-
Update your Docker image to
apache/airflow:3.0.0.๐ณ - Add dag-processor service to your Compose/Kubernetes manifests.๐งฉ
- Rename webserver โ api-server in service definitions.โ๏ธ
-
Set the three critical env vars:๐
AIRFLOW__CORE__EXECUTION_API_SERVER_URLAIRFLOW__API_AUTH__JWT_SECRET- SimpleAuthManager password config
-
Run
airflow db migrate.๐ - Update all import paths (use Ruff auto-fix first, then manual review).๐ ๏ธ
-
Update healthchecks to
airflow jobs check --local.๐ฉบ
Phase 3 โ Validateโ
# Check all services healthy
curl http://localhost:8080/api/v2/monitor/health | python3 -m json.tool
# Expected output
{
"metadatabase": {"status": "healthy"},
"scheduler": {"status": "healthy"},
"triggerer": {"status": "healthy"},
"dag_processor": {"status": "healthy"}
}
# Trigger a test DAG
airflow dags trigger your_test_dag
# Check task state
airflow tasks states-for-dag-run your_test_dag <run_id>
Part 10 โ Should You Upgrade?๐ค
Upgrade Now If:๐
- You are starting a new project โ there is no reason to build on Airflow 2.โจ
- You have simple DAGs (PythonOperator, BashOperator, standard providers) โ the migration is mostly find-and-replace on import paths.๐ ๏ธ
- You want DAG versioning โ this solves real operational pain.๐ฐ๏ธ
- You are running on Kubernetes โ the separation of concerns maps cleanly to individual pod scaling.๐๏ธ
Wait If:๐
- You depend heavily on FAB's OAuth/LDAP integrations and have not tested the FAB provider on Airflow 3.๐
- You have extensive SLA miss callback logic and no monitoring alternative ready.โฐ
- Your codebase has heavy direct metadata database access in task code โ refactoring that to the Python Client is non-trivial.๐๏ธ
- You use CeleryKubernetesExecutor or LocalKubernetesExecutor โ both are removed; you need to evaluate the Multiple Executor Configuration feature instead.๐งฉ
- You have custom Flask-AppBuilder views or blueprints โ these require porting to FastAPI.๐จ
The Honest Assessmentโ๏ธ
Airflow 3 is the version the project should have been architecturally from the beginning. The separation of the dag-processor, the Task Execution API, and the prohibition on direct metadata access are the right engineering decisions. They make Airflow significantly more secure, more scalable, and more maintainable at the cost of a one-time migration investment.๐ฆพ
The upgrade complexity is proportional to how much your codebase relied on Airflow 2's leaky abstractions: direct database access, FAB internals, SLA callbacks, and SubDAGs. If you followed Airflow 2 best practices (TaskFlow API, provider operators, no direct DB access), the migration is a half-day of import path updates and Docker Compose additions.๐ ๏ธ
If you did not, this upgrade is the forcing function to do it properly.๐
Conclusion๐
The jump from Airflow 2 to Airflow 3 is the most significant change in the project's history. The webserver is gone. The scheduler no longer parses DAGs. Tasks no longer touch the metadata database. The JWT-authenticated Execution API connects them all.๐
Each of these changes surfaces as a concrete failure mode in the first deployment: CPU spikes from JWT key divergence, Connection refused from wrong service URLs, silent healthcheck failures from removed ports, and silently no-op user creation from a replaced auth manager.๐งจ
Understanding the why behind the architecture โ isolation, security, scalability โ converts each failure from mysterious to obvious. The fixes are not workarounds; they are the intended configuration patterns for a distributed, multi-service orchestration system.๐ง
Airflow 3 is what a modern data orchestrator should look like. Migrate when you are ready, migrate properly, and you will not look back.๐
Resources
- Apache Airflow 3.0 Release Notes
- Official Upgrade to Airflow 3 Guide
- Astronomer: Upgrading from Airflow 2 to 3
- AIP-72: Task Execution Interface
- AIP-66: DAG Versioning
- Ruff AIR linting rules
Written from direct production experience migrating a healthcare ML retraining pipeline from Airflow 2 patterns to Airflow 3.0.0 on Docker Compose, April 2026 ๐.
Top comments (0)