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

[GitHub] [airflow] Lee-W commented on a diff in pull request #34592: Respect 'soft_fail' argument when running BatchSensors

Lee-W commented on code in PR #34592:
URL: https://github.com/apache/airflow/pull/34592#discussion_r1335726360


##########
tests/providers/amazon/aws/sensors/test_batch.py:
##########
@@ -242,3 +305,33 @@ def test_poke_invalid(self, mock_batch_client, batch_job_queue_sensor: BatchJobQ
             jobQueues=[JOB_QUEUE],
         )
         assert "AWS Batch job queue failed" in str(ctx.value)
+
+    @pytest.mark.parametrize(
+        "soft_fail, expected_exception", ((False, AirflowException), (True, AirflowSkipException))
+    )
+    @mock.patch.object(BatchClientHook, "client")
+    def test_fail_poke_no_jobqueues_status(
+        self, mock_batch_client, batch_job_queue_sensor: BatchJobQueueSensor, soft_fail, expected_exception
+    ):
+        job_queue = []
+        mock_batch_client.describe_job_queues.return_value = {"jobQueues": job_queue}
+        batch_job_queue_sensor.treat_non_existing_as_deleted = False
+        batch_job_queue_sensor.soft_fail = soft_fail
+        message = "AWS Batch job queue"
+        with pytest.raises(expected_exception, match=message):
+            batch_job_queue_sensor.poke({})
+
+    @pytest.mark.parametrize(
+        "soft_fail, expected_exception", ((False, AirflowException), (True, AirflowSkipException))
+    )
+    @mock.patch.object(BatchClientHook, "client")
+    def test_fail_poke_invalid_jobqueues_status(
+        self, mock_batch_client, batch_job_queue_sensor: BatchJobQueueSensor, soft_fail, expected_exception
+    ):
+        job_queue = [{"status": "UNKNOWN_STATUS"}]
+        mock_batch_client.describe_job_queues.return_value = {"jobQueues": job_queue}
+        batch_job_queue_sensor.treat_non_existing_as_deleted = False
+        batch_job_queue_sensor.soft_fail = soft_fail
+        message = "AWS Batch job queue"
+        with pytest.raises(expected_exception, match=message):
+            batch_job_queue_sensor.poke({})

Review Comment:
   ```suggestion
       @pytest.mark.parametrize(
           "soft_fail, expected_exception", ((False, AirflowException), (True, AirflowSkipException))
       )
       @pytest.mark.parametrize("job_queue", ([], [{"status": "UNKNOWN_STATUS"}]))
       @mock.patch.object(BatchClientHook, "client")
       def test_fail_poke(
           self, mock_batch_client, batch_job_queue_sensor: BatchJobQueueSensor, job_queue, soft_fail, expected_exception
       ):
           mock_batch_client.describe_job_queues.return_value = {"jobQueues": job_queue}
           batch_job_queue_sensor.treat_non_existing_as_deleted = False
           batch_job_queue_sensor.soft_fail = soft_fail
           message = "AWS Batch job queue"
           with pytest.raises(expected_exception, match=message):
               batch_job_queue_sensor.poke({})
   ```
   
   same above



##########
tests/providers/amazon/aws/sensors/test_batch.py:
##########
@@ -174,6 +201,42 @@ def test_poke_invalid(
         )
         assert "AWS Batch compute environment failed" in str(ctx.value)
 
+    @pytest.mark.parametrize(
+        "soft_fail, expected_exception", ((False, AirflowException), (True, AirflowSkipException))
+    )
+    @mock.patch.object(BatchClientHook, "client")
+    def test_fail_poke_no_compute_env(
+        self,
+        mock_batch_client,
+        batch_compute_environment_sensor: BatchComputeEnvironmentSensor,
+        soft_fail,
+        expected_exception,
+    ):
+        compute_env = []
+        mock_batch_client.describe_compute_environments.return_value = {"computeEnvironments": compute_env}
+        message = "AWS Batch compute environment"
+        batch_compute_environment_sensor.soft_fail = soft_fail
+        with pytest.raises(expected_exception, match=message):
+            batch_compute_environment_sensor.poke({})
+
+    @pytest.mark.parametrize(
+        "soft_fail, expected_exception", ((False, AirflowException), (True, AirflowSkipException))
+    )
+    @mock.patch.object(BatchClientHook, "client")
+    def test_fail_poke_unknown_status(
+        self,
+        mock_batch_client,
+        batch_compute_environment_sensor: BatchComputeEnvironmentSensor,
+        soft_fail,
+        expected_exception,
+    ):
+        compute_env = [{"status": "unknown_status"}]
+        mock_batch_client.describe_compute_environments.return_value = {"computeEnvironments": compute_env}
+        message = "AWS Batch compute environment failed. AWS Batch compute environment status:"
+        batch_compute_environment_sensor.soft_fail = soft_fail
+        with pytest.raises(expected_exception, match=message):
+            batch_compute_environment_sensor.poke({})

Review Comment:
   ```suggestion
       @pytest.mark.parametrize(
           "soft_fail, expected_exception", ((False, AirflowException), (True, AirflowSkipException))
       )
       @pytest.mark.parametrize(
           "compute_env, error_message",
           (
               ([], "AWS Batch compute environment"),
               ([{"status": "unknown_status"}], "AWS Batch compute environment failed. AWS Batch compute environment status:")
           )
       )
       @mock.patch.object(BatchClientHook, "client")
       def test_fail_poke(
           self,
           mock_batch_client,
           batch_compute_environment_sensor: BatchComputeEnvironmentSensor,
           compute_env,
           error_message,
           soft_fail,
           expected_exception,
       ):
           compute_env = []
           mock_batch_client.describe_compute_environments.return_value = {"computeEnvironments": compute_env}
           message = 
           batch_compute_environment_sensor.soft_fail = soft_fail
           with pytest.raises(expected_exception, match=error_message):
               batch_compute_environment_sensor.poke({})
   ```
   
   Not sure whether it makes sense, but I might try something like this



-- 
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