Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix template rendered bucket_key in S3KeySensor #28340

Merged
merged 3 commits into from
Dec 14, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 5 additions & 2 deletions airflow/providers/amazon/aws/sensors/s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ def __init__(
):
super().__init__(**kwargs)
self.bucket_name = bucket_name
self.bucket_key = [bucket_key] if isinstance(bucket_key, str) else bucket_key
self.bucket_key = bucket_key
self.wildcard_match = wildcard_match
self.check_fn = check_fn
self.aws_conn_id = aws_conn_id
Expand Down Expand Up @@ -125,7 +125,10 @@ def _check_key(self, key):
return True

def poke(self, context: Context):
return all(self._check_key(key) for key in self.bucket_key)
if isinstance(self.bucket_key, str):
return self._check_key(self.bucket_key)
else:
return all(self._check_key(key) for key in self.bucket_key)

def get_hook(self) -> S3Hook:
"""Create and return an S3Hook"""
Expand Down
27 changes: 27 additions & 0 deletions tests/providers/amazon/aws/sensors/test_s3_key.py
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,33 @@ def test_parse_bucket_key_from_jinja(self, mock_head_object):

mock_head_object.assert_called_once_with("key", "bucket")

@mock.patch("airflow.providers.amazon.aws.sensors.s3.S3Hook.head_object")
def test_parse_list_of_bucket_keys_from_jinja(self, mock_head_object):
mock_head_object.return_value = None
mock_head_object.side_effect = [{"ContentLength": 0}, {"ContentLength": 0}]

Variable.set("test_bucket_key", ["s3://bucket/file1", "s3://bucket/file2"])

execution_date = timezone.datetime(2020, 1, 1)

dag = DAG("test_s3_key", start_date=execution_date, render_template_as_native_obj=True)
op = S3KeySensor(
task_id="s3_key_sensor",
bucket_key="{{ var.value.test_bucket_key }}",
bucket_name=None,
dag=dag,
)

dag_run = DagRun(dag_id=dag.dag_id, execution_date=execution_date, run_id="test")
ti = TaskInstance(task=op)
ti.dag_run = dag_run
context = ti.get_template_context()
ti.render_templates(context)
op.poke(None)

mock_head_object.assert_any_call("file1", "bucket")
mock_head_object.assert_any_call("file2", "bucket")

@mock.patch("airflow.providers.amazon.aws.sensors.s3.S3Hook.head_object")
def test_poke(self, mock_head_object):
op = S3KeySensor(task_id="s3_key_sensor", bucket_key="s3://test_bucket/file")
Expand Down