Uploaded image for project: 'Apache Airflow'
  1. Apache Airflow
  2. AIRFLOW-2771

S3Hook Broad Exception Silent Failure

    XMLWordPrintableJSON

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Minor
    • Resolution: Fixed
    • Affects Version/s: 1.9.0
    • Fix Version/s: 1.10.0, 1.10.2
    • Component/s: hooks
    • Labels:

      Description

      Scenario

      S3KeySensor is passed an invalid S3/AWS connection id name (doesn't exist or bad permissions). There are also no credentials found under ~/.aws/credentials for boto to fallback on.

       

      When poking for the key, it creates an S3Hook and calls `check_for_key` on the hook. If the call to HeadObject fails, the call is caught by a generic except clause that catches all exceptions, rather than the expected botocore.exceptions.ClientError when an object is not found.

      Problem

      This causes the sensor to return False and report no issue with the task instance until it times out, rather than intuitively failing immediately if the connection is incorrectly configured. The current logging output gives no insight as to why the key is not being found.

      Current code

      try:
          self.get_conn().head_object(Bucket=bucket_name, Key=key)
          return True
      except:  # <- This catches credential and connection exceptions that should be raised
          return False
      
      from airflow.hooks.S3_hook import S3Hook
      hook = S3Hook(aws_conn_id="conn_that_doesnt_exist")
      hook.check_for_key(key="test", bucket="test")
      False
      
      [2018-07-18 18:57:26,652] {base_task_runner.py:98} INFO - Subtask: [2018-07-18 18:57:26,651] {sensors.py:537} INFO - Poking for key : s3://bucket/key.txt
      [2018-07-18 18:57:26,681] {base_task_runner.py:98} INFO - Subtask: [2018-07-18 18:57:26,680] {connectionpool.py:735} INFO - Starting new HTTPS connection (1): bucket.s3.amazonaws.com
      [2018-07-18 18:58:26,767] {base_task_runner.py:98} INFO - Subtask: [2018-07-18 18:58:26,767] {sensors.py:537} INFO - Poking for key : s3://bucket/key.txt
      [2018-07-18 18:58:26,809] {base_task_runner.py:98} INFO - Subtask: [2018-07-18 18:58:26,808] {connectionpool.py:735} INFO - Starting new HTTPS connection (1): bucket.s3.amazonaws.com
      

      Expected

      No credentials
      from airflow.hooks.S3_hook import S3Hook
      hook = S3Hook(aws_conn_id="conn_that_doesnt_exist")
      hook.check_for_key(key="test", bucket="test")
      Traceback (most recent call last):
      ...
      botocore.exceptions.NoCredentialsError: Unable to locate credentials
      
      Good credentials
      from airflow.hooks.S3_hook import S3Hook
      hook = S3Hook(aws_conn_id="conn_that_does_exist")
      hook.check_for_key(key="test", bucket="test")
      False
      

      Proposed Change

      Add a type to the except clause for botocore.exceptions.ClientError and log the message for both check_for_key and check_for_bucket on S3Hook.

      try:
          self.get_conn().head_object(Bucket=bucket_name, Key=key)
          return True
      except ClientError as e:
          self.log.info(e.response["Error"]["Message"]) 
          return False
      

        

        Attachments

          Activity

            People

            • Assignee:
              mascah Micheal Ascah
              Reporter:
              mascah Micheal Ascah
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved: