Code Monkey home page Code Monkey logo

Comments (13)

PhilippDB avatar PhilippDB commented on June 24, 2024 11

@austin-phil , @tylerwmarrs: Starting with v2.2.0, there is a foreign key constraint in TaskInstance and TaskReschedule referencing the DagRun with the "on_delete" option set to "CASCADE". Deleting a row in the DagRun table should therefore delete the corresponding rows in the TaskInstance and TaskReschedule tables automatically. At least if you're using a database backend that supports this such as Postgres.

So simply removing TaskInstance and TaskReschedule from the DATABASE_OBJECTS list should yield the desired behavior.

References:
https://airflow.apache.org/docs/apache-airflow/2.2.5/_modules/airflow/models/taskinstance.html#TaskInstance
https://airflow.apache.org/docs/apache-airflow/2.2.5/_modules/airflow/models/taskreschedule.html#TaskReschedule

from airflow-maintenance-dags.

austin-phil avatar austin-phil commented on June 24, 2024 10

For anyone looking at this on v2.2.4 or greater, you also need to change the "age_check_column" field within the db object for TaskInstance and TaskReschedule to a DateTime column otherwise the cleanup won't work properly - I made the following changes to both instances and it looks like everything is running properly now:

##### Change 1:
        'TaskInstance' : {
            "airflow_db_model": TaskInstance,
            "age_check_column": TaskInstance.start_date, #changed
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }
##### Change 2:
    try:
        from airflow.models import TaskReschedule
        DATABASE_OBJECTS_DICTS['TaskReschedule'] = {
            "airflow_db_model": TaskReschedule,
            "age_check_column": TaskReschedule.start_date, # changed
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

from airflow-maintenance-dags.

ud803 avatar ud803 commented on June 24, 2024 7

I'm facing similar issues after migrating from 2.1.2 to 2.2.1
And here's how I detoured it. Not sure if it works okay, use with caution.

Change 1

Turn DATABASE_OBJECTS into lists of names

DATABASE_OBJECTS = ['BaseJob', 'DagRun', 'TaskInstance', 'Log', 'XCom', 'SlaMiss', 'DagModel', 'TaskReschedule', 'TaskFail', 'RenderedTaskInstanceFields', 'ImportError', 'Task', 'TaskSet']

Change 2

include database_objects_dicts inside the cleanup_function, as it raises error when Database_objects are passed as arguments.

You SHOULD CHANGE execution_date variable to run_id of TaskReschedule and TaskInstance as below. Check Here for the Field name changes

def cleanup_function(**context):

    DATABASE_OBJECTS_DICTS = {
        'BaseJob' : {
            "airflow_db_model": BaseJob,
            "age_check_column": BaseJob.latest_heartbeat,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'DagRun' : {
            "airflow_db_model": DagRun,
            "age_check_column": DagRun.execution_date,
            "keep_last": True,
            "keep_last_filters": [DagRun.external_trigger.is_(False)],
            "keep_last_group_by": DagRun.dag_id
        },
        'TaskInstance' : {
            "airflow_db_model": TaskInstance,
            "age_check_column": TaskInstance.run_id,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'Log' : {
            "airflow_db_model": Log,
            "age_check_column": Log.dttm,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'XCom' : {
            "airflow_db_model": XCom,
            "age_check_column": XCom.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'SlaMiss' : {
            "airflow_db_model": SlaMiss,
            "age_check_column": SlaMiss.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'DagModel' : {
            "airflow_db_model": DagModel,
            "age_check_column": dag_model_last_scheduler_run,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }
    }

    # Check for TaskReschedule model
    try:
        from airflow.models import TaskReschedule
        DATABASE_OBJECTS_DICTS['TaskReschedule'] = {
            "airflow_db_model": TaskReschedule,
            "age_check_column": TaskReschedule.run_id,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for TaskFail model
    try:
        from airflow.models import TaskFail
        DATABASE_OBJECTS_DICTS['TaskFail'] = {
            "airflow_db_model": TaskFail,
            "age_check_column": TaskFail.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for RenderedTaskInstanceFields model
    try:
        from airflow.models import RenderedTaskInstanceFields
        DATABASE_OBJECTS_DICTS['RenderedTaskInstanceFields'] = {
            "airflow_db_model": RenderedTaskInstanceFields,
            "age_check_column": RenderedTaskInstanceFields.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for ImportError model
    try:
        from airflow.models import ImportError
        DATABASE_OBJECTS_DICTS['ImportError'] = {
            "airflow_db_model": ImportError,
            "age_check_column": ImportError.timestamp,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for celery executor
    airflow_executor = str(conf.get("core", "executor"))
    logging.info("Airflow Executor: " + str(airflow_executor))
    if(airflow_executor == "CeleryExecutor"):
        logging.info("Including Celery Modules")
        try:
            from celery.backends.database.models import Task, TaskSet
            DATABASE_OBJECTS_DICTS['Task'] = {
                    "airflow_db_model": Task,
                    "age_check_column": Task.date_done,
                    "keep_last": False,
                    "keep_last_filters": None,
                    "keep_last_group_by": None
            }
            DATABASE_OBJECTS_DICTS['TaskSet'] = {
                    "airflow_db_model": TaskSet,
                    "age_check_column": TaskSet.date_done,
                    "keep_last": False,
                    "keep_last_filters": None,
                    "keep_last_group_by": None
            }

        except Exception as e:
            logging.error(e)

Change 3

And change the task_id part and params part.

for db_object in DATABASE_OBJECTS:

    cleanup_op = PythonOperator(
        task_id='cleanup_' + str(db_object),
        python_callable=cleanup_function,
        params={'object_name' : db_object},
        dag=dag
    )

    print_configuration.set_downstream(cleanup_op)

It does not raises error anymore. But it does not guarantee performance or other stabilities. I'm just detouring it to make it work.

from airflow-maintenance-dags.

alionar avatar alionar commented on June 24, 2024 7

i tested @ealebed and @ud803 fix for db-cleanup in Airflow 2.2.1 and now JSON serialize error are now gone
thanks :)

success

"""
A maintenance workflow that you can deploy into Airflow to periodically clean
out the DagRun, TaskInstance, Log, XCom, Job DB and SlaMiss entries to avoid
having too much data in your Airflow MetaStore.

airflow trigger_dag --conf '[curly-braces]"maxDBEntryAgeInDays":30[curly-braces]' airflow-db-cleanup

--conf options:
    maxDBEntryAgeInDays:<INT> - Optional

"""
import airflow
from airflow import settings
from airflow.configuration import conf
from airflow.models import DAG, DagModel, DagRun, Log, XCom, SlaMiss, TaskInstance, Variable
try:
    from airflow.jobs import BaseJob
except Exception as e:
    from airflow.jobs.base_job import BaseJob
from airflow.operators.python_operator import PythonOperator
from datetime import datetime, timedelta
import dateutil.parser
import logging
import os
from sqlalchemy import func, and_
from sqlalchemy.exc import ProgrammingError
from sqlalchemy.orm import load_only

try:
    # airflow.utils.timezone is available from v1.10 onwards
    from airflow.utils import timezone
    now = timezone.utcnow
except ImportError:
    now = datetime.utcnow

# airflow-db-cleanup
DAG_ID = os.path.basename(__file__).replace(".pyc", "").replace(".py", "")
START_DATE = airflow.utils.dates.days_ago(1)
# How often to Run. @daily - Once a day at Midnight (UTC)
SCHEDULE_INTERVAL = "@daily"
# Who is listed as the owner of this DAG in the Airflow Web Server
DAG_OWNER_NAME = "operations"
# List of email address to send email alerts to if this job fails
ALERT_EMAIL_ADDRESSES = []
# Length to retain the log files if not already provided in the conf. If this
# is set to 30, the job will remove those files that arE 30 days old or older.

DEFAULT_MAX_DB_ENTRY_AGE_IN_DAYS = int(
    Variable.get("airflow_db_cleanup__max_db_entry_age_in_days", 30)
)
# Prints the database entries which will be getting deleted; set to False to avoid printing large lists and slowdown process
PRINT_DELETES = True
# Whether the job should delete the db entries or not. Included if you want to
# temporarily avoid deleting the db entries.
ENABLE_DELETE = True

# get dag model last schedule run
try:
    dag_model_last_scheduler_run = DagModel.last_scheduler_run
except AttributeError:
    dag_model_last_scheduler_run = DagModel.last_parsed_time

# List of all the objects that will be deleted. Comment out the DB objects you
# want to skip.
DATABASE_OBJECTS = ['BaseJob', 'DagRun', 'TaskInstance', 'Log', 'XCom', 'SlaMiss', 'DagModel', 'TaskReschedule', 'TaskFail', 'RenderedTaskInstanceFields', 'ImportError', 'Task', 'TaskSet']

session = settings.Session()

default_args = {
    'owner': DAG_OWNER_NAME,
    'depends_on_past': False,
    'email': ALERT_EMAIL_ADDRESSES,
    'email_on_failure': True,
    'email_on_retry': False,
    'start_date': START_DATE,
    'retries': 1,
    'retry_delay': timedelta(minutes=1)
}

dag = DAG(
    DAG_ID,
    default_args=default_args,
    schedule_interval=SCHEDULE_INTERVAL,
    start_date=START_DATE,
    tags=['airflow-maintenance-dags']
)
if hasattr(dag, 'doc_md'):
    dag.doc_md = __doc__
if hasattr(dag, 'catchup'):
    dag.catchup = False


def print_configuration_function(**context):
    logging.info("Loading Configurations...")
    dag_run_conf = context.get("dag_run").conf
    logging.info("dag_run.conf: " + str(dag_run_conf))
    max_db_entry_age_in_days = None
    if dag_run_conf:
        max_db_entry_age_in_days = dag_run_conf.get(
            "maxDBEntryAgeInDays", None
        )
    logging.info("maxDBEntryAgeInDays from dag_run.conf: " + str(dag_run_conf))
    if (max_db_entry_age_in_days is None or max_db_entry_age_in_days < 1):
        logging.info(
            "maxDBEntryAgeInDays conf variable isn't included or Variable " +
            "value is less than 1. Using Default '" +
            str(DEFAULT_MAX_DB_ENTRY_AGE_IN_DAYS) + "'"
        )
        max_db_entry_age_in_days = DEFAULT_MAX_DB_ENTRY_AGE_IN_DAYS
    max_date = now() + timedelta(-max_db_entry_age_in_days)
    logging.info("Finished Loading Configurations")
    logging.info("")

    logging.info("Configurations:")
    logging.info("max_db_entry_age_in_days: " + str(max_db_entry_age_in_days))
    logging.info("max_date:                 " + str(max_date))
    logging.info("enable_delete:            " + str(ENABLE_DELETE))
    logging.info("session:                  " + str(session))
    logging.info("")

    logging.info("Setting max_execution_date to XCom for Downstream Processes")
    context["ti"].xcom_push(key="max_date", value=max_date.isoformat())


print_configuration = PythonOperator(
    task_id='print_configuration',
    python_callable=print_configuration_function,
    provide_context=True,
    dag=dag)


def cleanup_function(**context):

    DATABASE_OBJECTS_DICTS = {
        'BaseJob' : {
            "airflow_db_model": BaseJob,
            "age_check_column": BaseJob.latest_heartbeat,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'DagRun' : {
            "airflow_db_model": DagRun,
            "age_check_column": DagRun.execution_date,
            "keep_last": True,
            "keep_last_filters": [DagRun.external_trigger.is_(False)],
            "keep_last_group_by": DagRun.dag_id
        },
        'TaskInstance' : {
            "airflow_db_model": TaskInstance,
            "age_check_column": TaskInstance.run_id,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'Log' : {
            "airflow_db_model": Log,
            "age_check_column": Log.dttm,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'XCom' : {
            "airflow_db_model": XCom,
            "age_check_column": XCom.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'SlaMiss' : {
            "airflow_db_model": SlaMiss,
            "age_check_column": SlaMiss.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        },
        'DagModel' : {
            "airflow_db_model": DagModel,
            "age_check_column": dag_model_last_scheduler_run,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }
    }

    # Check for TaskReschedule model
    try:
        from airflow.models import TaskReschedule
        DATABASE_OBJECTS_DICTS['TaskReschedule'] = {
            "airflow_db_model": TaskReschedule,
            "age_check_column": TaskReschedule.run_id,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for TaskFail model
    try:
        from airflow.models import TaskFail
        DATABASE_OBJECTS_DICTS['TaskFail'] = {
            "airflow_db_model": TaskFail,
            "age_check_column": TaskFail.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for RenderedTaskInstanceFields model
    try:
        from airflow.models import RenderedTaskInstanceFields
        DATABASE_OBJECTS_DICTS['RenderedTaskInstanceFields'] = {
            "airflow_db_model": RenderedTaskInstanceFields,
            "age_check_column": RenderedTaskInstanceFields.execution_date,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for ImportError model
    try:
        from airflow.models import ImportError
        DATABASE_OBJECTS_DICTS['ImportError'] = {
            "airflow_db_model": ImportError,
            "age_check_column": ImportError.timestamp,
            "keep_last": False,
            "keep_last_filters": None,
            "keep_last_group_by": None
        }

    except Exception as e:
        logging.error(e)

    # Check for celery executor
    airflow_executor = str(conf.get("core", "executor"))
    logging.info("Airflow Executor: " + str(airflow_executor))
    if(airflow_executor == "CeleryExecutor"):
        logging.info("Including Celery Modules")
        try:
            from celery.backends.database.models import Task, TaskSet
            DATABASE_OBJECTS_DICTS['Task'] = {
                    "airflow_db_model": Task,
                    "age_check_column": Task.date_done,
                    "keep_last": False,
                    "keep_last_filters": None,
                    "keep_last_group_by": None
            }
            DATABASE_OBJECTS_DICTS['TaskSet'] = {
                    "airflow_db_model": TaskSet,
                    "age_check_column": TaskSet.date_done,
                    "keep_last": False,
                    "keep_last_filters": None,
                    "keep_last_group_by": None
            }

        except Exception as e:
            logging.error(e)

    logging.info("Retrieving max_execution_date from XCom")
    max_date = context["ti"].xcom_pull(
        task_ids=print_configuration.task_id, key="max_date"
    )
    max_date = dateutil.parser.parse(max_date)  # stored as iso8601 str in xcom

    object_name = str(context["params"].get("object_name"))

    airflow_db_model = DATABASE_OBJECTS_DICTS[object_name].get("airflow_db_model")
    age_check_column = DATABASE_OBJECTS_DICTS[object_name].get("age_check_column")
    keep_last = DATABASE_OBJECTS_DICTS[object_name].get("keep_last")
    keep_last_filters = DATABASE_OBJECTS_DICTS[object_name].get("keep_last_filters")
    keep_last_group_by = DATABASE_OBJECTS_DICTS[object_name].get("keep_last_group_by")
    state = DATABASE_OBJECTS_DICTS[object_name].get("state")

    logging.info("Configurations:")
    logging.info("max_date:                 " + str(max_date))
    logging.info("enable_delete:            " + str(ENABLE_DELETE))
    logging.info("session:                  " + str(session))
    logging.info("airflow_db_model:         " + str(airflow_db_model))
    logging.info("state:                    " + str(state))
    logging.info("age_check_column:         " + str(age_check_column))
    logging.info("keep_last:                " + str(keep_last))
    logging.info("keep_last_filters:        " + str(keep_last_filters))
    logging.info("keep_last_group_by:       " + str(keep_last_group_by))

    logging.info("")

    logging.info("Running Cleanup Process...")

    try:
        query = session.query(airflow_db_model).options(
            load_only(age_check_column)
        )

        logging.info("INITIAL QUERY : " + str(query))

        if keep_last:

            subquery = session.query(func.max(DagRun.execution_date))
            # workaround for MySQL "table specified twice" issue
            # https://github.com/teamclairvoyant/airflow-maintenance-dags/issues/41
            if keep_last_filters is not None:
                for entry in keep_last_filters:
                    subquery = subquery.filter(entry)

                logging.info("SUB QUERY [keep_last_filters]: " + str(subquery))

            if keep_last_group_by is not None:
                subquery = subquery.group_by(keep_last_group_by)
                logging.info(
                    "SUB QUERY [keep_last_group_by]: " + str(subquery))

            subquery = subquery.from_self()

            query = query.filter(
                and_(age_check_column.notin_(subquery)),
                and_(age_check_column <= max_date)
            )

        else:
            query = query.filter(age_check_column <= max_date,)

        if PRINT_DELETES:
            entries_to_delete = query.all()

            logging.info("Query: " + str(query))
            logging.info(
                "Process will be Deleting the following " +
                str(airflow_db_model.__name__) + "(s):"
            )
            for entry in entries_to_delete:
                logging.info(
                    "\tEntry: " + str(entry) + ", Date: " +
                    str(entry.__dict__[str(age_check_column).split(".")[1]])
                )

            logging.info(
                "Process will be Deleting " + str(len(entries_to_delete)) + " " +
                str(airflow_db_model.__name__) + "(s)"
            )
        else:
            logging.warn(
                "You've opted to skip printing the db entries to be deleted. Set PRINT_DELETES to True to show entries!!!")

        if ENABLE_DELETE:
            logging.info("Performing Delete...")
            # using bulk delete
            query.delete(synchronize_session=False)
            session.commit()
            logging.info("Finished Performing Delete")
        else:
            logging.warn(
                "You've opted to skip deleting the db entries. Set ENABLE_DELETE to True to delete entries!!!")

        logging.info("Finished Running Cleanup Process")

    except ProgrammingError as e:
        logging.error(e)
        logging.error(str(airflow_db_model) +
                      " is not present in the metadata. Skipping...")


for db_object in DATABASE_OBJECTS:

    cleanup_op = PythonOperator(
        task_id='cleanup_' + str(db_object),
        python_callable=cleanup_function,
        params={'object_name' : db_object},
        dag=dag
    )

    print_configuration.set_downstream(cleanup_op)

from airflow-maintenance-dags.

OneMintJulep avatar OneMintJulep commented on June 24, 2024 6

Spent a while troubleshooting this as well. Issue is that the database objects being passed to the cleanup task via params aren't JSON serializable. This might've been introduced by apache/airflow#17100.

from airflow-maintenance-dags.

ealebed avatar ealebed commented on June 24, 2024 4

Great! Thanks @ud803!

Also, don't forget change:

    airflow_db_model = context["params"].get("airflow_db_model")
    state = context["params"].get("state")
    age_check_column = context["params"].get("age_check_column")
    keep_last = context["params"].get("keep_last")
    keep_last_filters = context["params"].get("keep_last_filters")
    keep_last_group_by = context["params"].get("keep_last_group_by")

to

    object_name = str(context["params"].get("object_name"))

    airflow_db_model = DATABASE_OBJECTS_DICTS[object_name].get("airflow_db_model")
    age_check_column = DATABASE_OBJECTS_DICTS[object_name].get("age_check_column")
    keep_last = DATABASE_OBJECTS_DICTS[object_name].get("keep_last")
    keep_last_filters = DATABASE_OBJECTS_DICTS[object_name].get("keep_last_filters")
    keep_last_group_by = DATABASE_OBJECTS_DICTS[object_name].get("keep_last_group_by")

from airflow-maintenance-dags.

Jaroslaw-Rachwalski avatar Jaroslaw-Rachwalski commented on June 24, 2024 4

It looks like that @PhilippDB has right regarding the constraints in Postgresql. The same situation is also for cleanup_BaseXCom, cleanup_RenderedTaskInstanceFields.

References:
https://airflow.apache.org/docs/apache-airflow/stable/_modules/airflow/models/xcom.html#BaseXCom
https://airflow.apache.org/docs/apache-airflow/stable/_modules/airflow/models/renderedtifields.html#RenderedTaskInstanceFields

I made a simple google search and it looks like that main databases (mysql, oracle, microsoft sql server, postgresql) support the 'on_delete=CASCADE' option.

from airflow-maintenance-dags.

4andy avatar 4andy commented on June 24, 2024 3

This is the error I am seeing for this issue

Broken DAG: [/dags/airflow-db-cleanup.py] Traceback (most recent call last):
  File "xxx/lib/python3.9/json/encoder.py", line 257, in iterencode
    return _iterencode(o, 0)
  File "xxx/lib/python3.9/json/encoder.py", line 179, in default
    raise TypeError(f'Object of type {o.__class__.__name__} '
TypeError: Object of type InstrumentedAttribute is not JSON serializable

from airflow-maintenance-dags.

tylerwmarrs avatar tylerwmarrs commented on June 24, 2024 2

@austin-phil Using start_date deletes different instances as that is not when the instance actually executed. Following the logic to comply with 2.2.x and newer, I adjusted the following:

        if airflow_db_model == TaskInstance or airflow_db_model == TaskReschedule:
            query = session.query(airflow_db_model).options(
                joinedload('dag_run').load_only('execution_date')
            )
        else:
            query = session.query(airflow_db_model).options(
                load_only(age_check_column)
            )

The issue is that the execution_date is referenced differently and you must join to dag_run.

Alternatively, you can add an option called "load_only" and pass that through to the query in the cleanup function. Example:

DATABASE_OBJECTS = [
    {
        "airflow_db_model": BaseJob,
        "age_check_column": BaseJob.latest_heartbeat,
        "keep_last": False,
        "keep_last_filters": None,
        "keep_last_group_by": None,
        "load_only": load_only(BaseJob.latest_heartbeat)
    },
   {
        "airflow_db_model": TaskInstance,
        "age_check_column": TaskInstance.execution_date,
        "keep_last": False,
        "keep_last_filters": None,
        "keep_last_group_by": None,
        "load_only": joinedload('dag_run').load_only('execution_date')
    },
]

# in cleanup
load_only_option = context["params"].get("load_only")
query = session.query(airflow_db_model).options(load_only_option)

References:
https://github.com/teamclairvoyant/airflow-maintenance-dags/blob/master/db-cleanup/airflow-db-cleanup.py#L294
https://github.com/apache/airflow/blob/v2-2-stable/airflow/models/taskinstance.py#L410
https://stackoverflow.com/a/39553869
https://docs.sqlalchemy.org/en/14/orm/extensions/associationproxy.html

from airflow-maintenance-dags.

ealebed avatar ealebed commented on June 24, 2024

I see this error too...
Also, I tried to localize problem, and I can confirm that it occurs in this part:

for db_object in DATABASE_OBJECTS:
    cleanup_op = PythonOperator(
        task_id='cleanup_' + str(db_object["airflow_db_model"].__name__),
        python_callable=cleanup_function,
        params=db_object,
        provide_context=True,
        dag=dag)

    print_configuration.set_downstream(cleanup_op)

from airflow-maintenance-dags.

wong-codaio avatar wong-codaio commented on June 24, 2024

FWIW - This is the full stacktrace

ERROR [airflow.models.dagbag.DagBag] Failed to write serialized DAG: /usr/local/airflow/dags/airflow-db-cleanup.py
Traceback (most recent call last):
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/airflow/models/dagbag.py", line 591, in _serialize_dag_capturing_errors
    dag_was_updated = SerializedDagModel.write_dag(
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/airflow/utils/session.py", line 67, in wrapper
    return func(*args, **kwargs)
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/airflow/models/serialized_dag.py", line 136, in write_dag
    new_serialized_dag = cls(dag)
  File "<string>", line 4, in __init__
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/sqlalchemy/orm/state.py", line 433, in _initialize_instance
    manager.dispatch.init_failure(self, args, kwargs)
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/sqlalchemy/util/langhelpers.py", line 68, in __exit__
    compat.raise_(
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/sqlalchemy/util/compat.py", line 182, in raise_
    raise exception
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/sqlalchemy/orm/state.py", line 430, in _initialize_instance
    return manager.original_init(*mixed[1:], **kwargs)
  File "/root/.local/share/virtualenvs/airflow-8LJOcfRp/lib/python3.8/site-packages/airflow/models/serialized_dag.py", line 97, in __init__
    self.dag_hash = hashlib.md5(json.dumps(self.data, sort_keys=True).encode("utf-8")).hexdigest()
  File "/usr/local/lib/python3.8/json/__init__.py", line 234, in dumps
    return cls(
  File "/usr/local/lib/python3.8/json/encoder.py", line 199, in encode
    chunks = self.iterencode(o, _one_shot=True)
  File "/usr/local/lib/python3.8/json/encoder.py", line 257, in iterencode
    return _iterencode(o, 0)
  File "/usr/local/lib/python3.8/json/encoder.py", line 179, in default
    raise TypeError(f'Object of type {o.__class__.__name__} '
TypeError: Object of type InstrumentedAttribute is not JSON serializable

from airflow-maintenance-dags.

HamishPoole avatar HamishPoole commented on June 24, 2024

if anyone is working actively on this, or even vaguely thinking about it, please reach out.

Fixing this is basically my week this week and I'd love to have a natter and catch up with anyone working on the same problem.

from airflow-maintenance-dags.

Haapalaj avatar Haapalaj commented on June 24, 2024

One approach we were trying was to give those classes and attributes as literals and then at the end, trying to objectify them.
It basicly works, but is somehow still cumbersome to apply for all the cases.

E.g.:


DATABASE_OBJECTS = [
    {
        "airflow_db_model": 'airflow.jobs.base_job.BaseJob',
        "age_check_column": 'BaseJob.latest_heartbeat',

Then in cleanup_function trying to:

        airflow_db_model = objectify_module_class(airflow_db_model)
        age_check_column = objectify_class_attribute(airflow_db_model, age_check_column)

Objectify by python 'locate' and 'getattr', but doing this dynamically for all of those database_objects, would require some additional information.

from airflow-maintenance-dags.

Related Issues (20)

Recommend Projects

  • React photo React

    A declarative, efficient, and flexible JavaScript library for building user interfaces.

  • Vue.js photo Vue.js

    🖖 Vue.js is a progressive, incrementally-adoptable JavaScript framework for building UI on the web.

  • Typescript photo Typescript

    TypeScript is a superset of JavaScript that compiles to clean JavaScript output.

  • TensorFlow photo TensorFlow

    An Open Source Machine Learning Framework for Everyone

  • Django photo Django

    The Web framework for perfectionists with deadlines.

  • D3 photo D3

    Bring data to life with SVG, Canvas and HTML. 📊📈🎉

Recommend Topics

  • javascript

    JavaScript (JS) is a lightweight interpreted programming language with first-class functions.

  • web

    Some thing interesting about web. New door for the world.

  • server

    A server is a program made to process requests and deliver data to clients.

  • Machine learning

    Machine learning is a way of modeling and interpreting data that allows a piece of software to respond intelligently.

  • Game

    Some thing interesting about game, make everyone happy.

Recommend Org

  • Facebook photo Facebook

    We are working to build community through open source technology. NB: members must have two-factor auth.

  • Microsoft photo Microsoft

    Open source projects and samples from Microsoft.

  • Google photo Google

    Google ❤️ Open Source for everyone.

  • D3 photo D3

    Data-Driven Documents codes.