yt-dlp-dags/dags/ytdlp_mgmt_queue_add_urls.py
2025-07-18 17:17:19 +03:00

175 lines
7.9 KiB
Python

from airflow import DAG
from airflow.models.param import Param
from airflow.operators.python import PythonOperator
from airflow.providers.redis.hooks.redis import RedisHook
from airflow.utils.dates import days_ago
from airflow.exceptions import AirflowException
from datetime import timedelta
import logging
import redis # Import redis exceptions if needed
# Import utility functions
from utils.redis_utils import _get_redis_client
# Configure logging
logger = logging.getLogger(__name__)
# Default settings
DEFAULT_QUEUE_NAME = 'video_queue' # Default base name for the queue
DEFAULT_REDIS_CONN_ID = 'redis_default'
# --- Python Callables for Tasks ---
def add_urls_callable(**context):
"""Adds URLs from comma/newline separated input to the specified Redis inbox list."""
params = context['params']
redis_conn_id = params['redis_conn_id']
queue_name = params['queue_name']
inbox_queue = f"{queue_name}_inbox"
urls_input = params['urls']
if not urls_input or not isinstance(urls_input, str):
logger.warning("No URLs provided or 'urls' parameter is not a string. Nothing to add.")
return
# Process input: split by newline, then by comma, flatten, strip, and filter empty
urls_to_add = []
for line in urls_input.splitlines():
urls_to_add.extend(url.strip() for url in line.split(',') if url.strip())
# Remove duplicates while preserving order (optional, but good practice)
seen = set()
urls_to_add = [x for x in urls_to_add if not (x in seen or seen.add(x))]
if not urls_to_add:
logger.info("No valid URLs found after processing input. Nothing added.")
return
logger.info(f"Attempting to add {len(urls_to_add)} unique URLs to Redis list '{inbox_queue}' using connection '{redis_conn_id}'.")
try:
redis_client = _get_redis_client(redis_conn_id)
# Use rpush to add to the end of the list (FIFO behavior with lpop)
added_count = redis_client.rpush(inbox_queue, *urls_to_add)
logger.info(f"Successfully added {len(urls_to_add)} URLs to list '{inbox_queue}'. New list length: {added_count}.")
except Exception as e:
logger.error(f"Failed to add URLs to Redis list '{inbox_queue}': {e}", exc_info=True)
raise AirflowException(f"Failed to add URLs to Redis: {e}")
# Removed clear_queue_callable as this DAG focuses on adding and verifying
def check_status_callable(**context):
"""Checks the type and length/size of the specified Redis inbox key."""
# Access DAG run parameters directly from context['params']
dag_params = context['params']
redis_conn_id = dag_params['redis_conn_id']
# This DAG verifies the inbox queue, so we construct the name from the base name
queue_name = dag_params['queue_name']
queue_to_check = f"{queue_name}_inbox"
if not queue_name:
raise ValueError("DAG parameter 'queue_name' (base name) cannot be empty.")
logger.info(f"Attempting to check status of Redis key '{queue_to_check}' using connection '{redis_conn_id}'.")
try:
# Use the resolved redis_conn_id to get the client
redis_client = _get_redis_client(redis_conn_id)
# redis_client.type returns bytes (e.g., b'list', b'hash', b'none')
key_type_bytes = redis_client.type(queue_to_check)
key_type_str = key_type_bytes.decode('utf-8') # Decode to string
length = 0
if key_type_str == 'list':
length = redis_client.llen(queue_to_check)
logger.info(f"Redis list '{queue_to_check}' has {length} items.")
elif key_type_str == 'hash':
length = redis_client.hlen(queue_to_check)
logger.info(f"Redis hash '{queue_to_check}' has {length} fields.")
elif key_type_str == 'none': # Check against the decoded string 'none'
logger.info(f"Redis key '{queue_to_check}' does not exist.")
else:
# Attempt to get size for other types if possible, e.g., set size
try:
if key_type_str == 'set':
length = redis_client.scard(queue_to_check)
logger.info(f"Redis set '{queue_to_check}' has {length} members.")
# Add checks for other types like zset if needed
else:
logger.info(f"Redis key '{queue_to_check}' exists but is of unhandled type '{key_type_str}'. Cannot determine size.")
except Exception as size_error:
logger.warning(f"Could not determine size for Redis key '{queue_to_check}' (type: {key_type_str}): {size_error}")
logger.info(f"Redis key '{queue_to_check}' exists but is of unhandled/unsizeable type '{key_type_str}'.")
# Push results to XCom
context['task_instance'].xcom_push(key='queue_key_type', value=key_type_str)
context['task_instance'].xcom_push(key='queue_size', value=length)
# Return status info using the resolved queue_to_check
return {'key': queue_to_check, 'type': key_type_str, 'size': length}
except Exception as e:
# Log error using the resolved queue_to_check
logger.error(f"Failed to check status of Redis key '{queue_to_check}': {e}", exc_info=True)
raise AirflowException(f"Failed to check Redis key status: {e}")
# --- DAG Definition ---
default_args = {
'owner': 'airflow',
'depends_on_past': False,
'email_on_failure': False,
'email_on_retry': False,
'retries': 1,
'retry_delay': timedelta(minutes=1), # Slightly longer retry delay for management tasks
'start_date': days_ago(1)
}
# This single DAG contains operators for different management actions,
# This DAG allows adding URLs and then checking the status of the target queue.
with DAG(
dag_id='ytdlp_mgmt_queue_add_and_verify', # Updated DAG ID
default_args=default_args,
schedule_interval=None, # Manually triggered
catchup=False,
description='Manually add URLs to a YTDLP inbox queue and verify the queue status.', # Updated description
tags=['ytdlp', 'queue', 'management', 'redis', 'manual', 'add', 'verify'], # Updated tags
params={
# Common params
'redis_conn_id': Param(DEFAULT_REDIS_CONN_ID, type="string", description="Airflow Redis connection ID."),
# Params for adding URLs (and checking the same queue)
'queue_name': Param(DEFAULT_QUEUE_NAME, type="string", title="Base Queue Name", description="Base name for the Redis queues (e.g., 'video_queue'). The DAG will add URLs to '{base_name}_inbox'."),
'urls': Param("", type="string", title="URLs to Add", description="Comma and/or newline separated list of video URLs.", multiline=True), # Updated description, keep multiline for UI
# Removed clear_queue_name param
# Removed check_queue_name param (will use queue_name)
}
) as dag:
add_urls_task = PythonOperator(
task_id='add_urls_to_queue',
python_callable=add_urls_callable,
# Pass only relevant params to the callable via context['params']
# Note: context['params'] automatically contains all DAG params
)
add_urls_task.doc_md = """
### Add URLs to Queue
Adds URLs from the `urls` parameter (comma/newline separated) to the Redis list specified by `queue_name`.
*Trigger this task manually via the UI and provide the URLs.*
"""
# Removed clear_queue_task
check_status_task = PythonOperator(
task_id='check_queue_status_after_add',
python_callable=check_status_callable,
# No task-specific params needed; callable uses context['params'] directly.
)
check_status_task.doc_md = """
### Check Queue Status After Add
Checks the type and length/size of the Redis key specified by `queue_name` (the same queue URLs were added to).
Logs the result and pushes `queue_key_type` and `queue_size` to XCom.
*This task runs automatically after `add_urls_to_queue`.*
"""
# Define dependency: Add URLs first, then check status
add_urls_task >> check_status_task