You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "jplauri (via GitHub)" <gi...@apache.org> on 2023/12/13 11:29:50 UTC

[I] Glue job not updating on iam_role_arn change GlueJobOperator/GlueJobHook [airflow]

jplauri opened a new issue, #36203:
URL: https://github.com/apache/airflow/issues/36203

   ### Apache Airflow version
   
   2.7.3
   
   ### What happened
   
   I'm running a DAG like:
   
   ```
   with DAG(
           dag_id="my_dag",
           default_args={
               "depends_on_past": False,
               "email_on_failure": False,
               "email_on_retry": False,
               "retries": 0,
               "retry_delay": timedelta(minutes=5)
           },
           start_date=datetime(2023, 12, 6),
           catchup=False
   ) as dag:
       start = EmptyOperator(task_id="start")
   
       etl_scripts_bucket = "my-etl-scripts"
       create_etl_script_bucket = S3CreateBucketOperator(
           task_id="create_bucket",
           bucket_name=etl_scripts_bucket
       )
   
       etl_job = GlueJobOperator(
           task_id="get_stuff",
           job_name="get_stuff",
           script_location="s3://my-airflow-bucket/dags/my_dag/test_etl.py",
           s3_bucket=etl_scripts_bucket,
           iam_role_arn="a-valid-iam-role-arn" # Consider this line
           create_job_kwargs={"GlueVersion": "4.0", "NumberOfWorkers": 2, "WorkerType": "G.1X"},
       )
   
       end = EmptyOperator(task_id="end")
   
       start >> create_etl_script_bucket >> etl_job >> end
   ```
   
   If `a-valid-iam-role-arn` is indeed valid, the DAG runs fine and it created a Glue job with the name `get_stuff`. Now, suppose that `iam_role_arn` changes, maybe I even break it on purpose, e.g., replace the line:
   
   `iam_role_arn="BREAK_ON_PURPOSE" # Consider this line`
   
   I would definitely expect the DAG to update the Glue job `get_stuff` or at least try to do so - if the role is completely bogus maybe it should break loudly or if the role ARN is valid but has insufficient permissions or whatever, the change should go through. After committing the change, re-running the DAG succeeds. Inspecting the Glue job from the AWS console still shows the IAM role to be `a-valid-iam-role-arn`, i.e., the job did not update.
   
   So while the DAG code does get updated, the Glue job does not. I observe the same effect with a completely bogus IAM role ARN (i.e., invalid ARN), an existing but semantically wrong ARN, and also with a correct, desired ARN that does exist and has the permissions to run whatever is needed.
   
   ### What you think should happen instead
   
   The Glue job should update with the new IAM role ARN.
   
   ### How to reproduce
   
   Have a valid default `aws_default` connection. Use the following DAG:
   
   ```
   from datetime import datetime
   
   from airflow import DAG
   from airflow.operators.empty import EmptyOperator
   from airflow.providers.amazon.aws.operators.glue import GlueJobOperator
   from airflow.providers.amazon.aws.operators.s3 import S3CreateBucketOperator
   
   with DAG(
           dag_id="bug_dag",
           start_date=datetime(2023, 12, 6),
           catchup=False
   ) as dag:
       start = EmptyOperator(task_id="start")
   
       etl_scripts_bucket = "my-dag-get-etl-scripts"
       create_etl_script_bucket = S3CreateBucketOperator(
           task_id="create_bucket",
           bucket_name=etl_scripts_bucket
       )
   
       # Perform these steps:
       # 1. Manually create a bucket with the name BUCKET_NAME.
       # 2. Add a file called "test_etl.py" with the contents: print('hello')
       # 3. Create an IAM role or use an existing one. Doesn't matter what permissions it has. Let the role ARN be ROLE_ARN.
       BUCKET_NAME = "REPLACE-ME"
       ROLE_ARN = "REPLACE-ME"
   
       etl_job = GlueJobOperator(
           task_id="do_etl",
           job_name="do_etl",
           script_location=f"s3://{BUCKET_NAME}/test_etl.py",
           s3_bucket=etl_scripts_bucket,
           iam_role_arn=ROLE_ARN,
           create_job_kwargs={"GlueVersion": "4.0", "NumberOfWorkers": 2, "WorkerType": "G.1X"},
       )
   
       end = EmptyOperator(task_id="end")
   
       start >> create_etl_script_bucket >> etl_job >> end
   ```
   
   Follow the three manual steps above (create bucket, add mock file, pick/create a new IAM role). Run the operator. Update the `iam_role_arn`. Run the operator. Notice now that the Glue job which has been created has not been updated.
   
   ### Operating System
   
   Amazon Linux 2023
   
   ### Versions of Apache Airflow Providers
   
   apache-airflow-providers-amazon==8.11.0
   
   ### Deployment
   
   Virtualenv installation
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   Looking at the code, the issue seems to be within `GlueJobHook`. Maybe the culprit is the `create_or_update_glue_job` method which in turn uses `create_glue_job_config`. I can't, however, see that anything would obviously be wrong.
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [I] Glue job not updating on iam_role_arn change GlueJobOperator/GlueJobHook [airflow]

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala closed issue #36203: Glue job not updating on iam_role_arn change GlueJobOperator/GlueJobHook
URL: https://github.com/apache/airflow/issues/36203


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [I] Glue job not updating on iam_role_arn change GlueJobOperator/GlueJobHook [airflow]

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala commented on issue #36203:
URL: https://github.com/apache/airflow/issues/36203#issuecomment-1859214239

   I just checked the code source for this operator and its hook. Did you try to provide `update_config=True` to your operator?
   > :param update_config: If True, Operator will update job configuration.  (default: False)
   
   In the hook, if it's True, the hook will create or update the job configuration, but if it's False (the default value), the hook will create the job or get it if it exists without updating the configuration.
   ```python
               if self.update_config:
                   job_name = self.create_or_update_glue_job()
               else:
                   job_name = self.get_or_create_glue_job()
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org