You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/05/16 19:38:00 UTC

[GitHub] [airflow] mustafagok opened a new pull request #8888: Add AWS ECS system test

mustafagok opened a new pull request #8888:
URL: https://github.com/apache/airflow/pull/8888


   Add AWS ECS system test
   
   ECS System Test to run and test example ECS dags
   
   Required variables.env file content (from your account):
   ```
   # Auto-export all variables
   set -a
   
   # aws parameters
   REGION_NAME="eu-west-1"
   REGISTRY_ID="123456789012"
   REPOSITORY_NAME="foobar/hello-world"
   SUBNET_ID="subnet-068e9654a3c357a"
   SECURITY_GROUP_ID="sg-054dc69874a651"
   EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
   
   # remove all created/existing resources flag
   # comment out to keep resources or use empty string
   # REMOVE_RESOURCES="True"
   ```
   @feluelle @xinbinhuang @potiuk @mik-laj @ashb ready for review :)
   
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Target Github ISSUE in description if exists
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   


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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426193936



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,140 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AmazonSystemTest, AWS_DAG_FOLDER
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        REPOSITORY_NAME="foobar/hello-world"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b/hello-world-container"
+
+    local_image_tag = "hello"
+    local_image_context_path = "/opt/airflow/tests/providers/amazon/aws/images/ecs_test_image/"
+
+    def setUp(self):
+        print("setUp")
+        super().setUp()
+        self.create_connection(
+            aws_conn_id=self.aws_conn_id,
+            region=self._region_name(),
+        )
+
+        # create repository in ecr if it does not exist
+        repository_exists, repository_uri = self.is_ecr_repository_exists(
+            aws_conn_id=self.aws_conn_id,
+            repository_name=self._repository_name(),
+            registry_id=self._registry_id(),
+        )
+        print(f"repository_exists: {repository_exists}")
+        if not repository_exists:
+            repository_uri = self.create_ecr_repository(
+                aws_conn_id=self.aws_conn_id,
+                repository_name=self._repository_name(),
+            )
+            print(f"repository_uri: {repository_uri}")
+        full_image_name = f"{repository_uri}:latest"
+
+        # prepare (build, tag, push) image
+        self.authenticate_client_to_ecr(
+            region=self._region_name(),
+        )
+        self.build_image(
+            tag=self.local_image_tag,
+            path=self.local_image_context_path,
+        )
+        self.tag_image(
+            source=f"{self.local_image_tag}:latest",
+            target=full_image_name,
+        )
+        self.push_image(
+            tag=full_image_name,
+        )

Review comment:
       Can we use public images from DockerHub for the system test?




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r436290678



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       done, can you check it please?

##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,96 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+

Review comment:
       you are right it should work but I didn't test it with `sqlite`
   
   :heavy_check_mark: done, I have added mark-backend

##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       thank you for your review




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426192991



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
       `is_ecr_repository_exists` is not intuitive because it does not tell that the function would also return a uri.




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r432724181



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       @xinbinhuang actually I don't understand what you mean




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r432993143



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       Either would work, the latter is recognized by Unit Test only, and both are recognized by [`pytest`](https://docs.pytest.org/en/latest/xunit_setup.html).




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r429579522



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       ```suggestion
       @classmethod
       def setup_class(cls):
   ```
   Small NIT: maybe use `setup_class` is more appropriate here, in case in the future we would have more than one dags run or other task cases. Same to `teardown`.




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r436246002



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,96 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+

Review comment:
       ```suggestion
   @pytest.mark.backend("mysql", "postgres")
   ```
   See also: https://github.com/apache/airflow/blob/master/TESTING.rst#running-backend-specific-tests
   
   Actually I think I do not quite understand the backend marker myself. I am wondering why for example do we use `@pytest.mark.backend("mysql", "postgres")` [here](https://github.com/apache/airflow/blob/b6f919cbc5f6784ee119da8f7a34348514f09d70/tests/providers/google/cloud/operators/test_automl_system.py#L24). What does this example dag have against running on `sqlite`? Why do we need the marker here?
   
   Maybe @turbaszek knows more about it :)




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426209091



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -51,6 +54,27 @@ def provide_aws_s3_bucket(name):
 @pytest.mark.system("amazon")
 class AmazonSystemTest(SystemTest):
 
+    @staticmethod
+    def _region_name():
+        return os.environ.get("REGION_NAME")
+
+    @staticmethod
+    def _registry_id():
+        return os.environ.get("REGISTRY_ID")
+
+    @staticmethod
+    def _repository_name():
+        return os.environ.get("REPOSITORY_NAME")
+
+    @staticmethod
+    def _execution_role_arn():
+        return os.environ.get("EXECUTION_ROLE_ARN")

Review comment:
       `@property` is not necessarily used to return `self._x`. IMHO, `@property` fits better here because the names and functionalities are pretty much attributes access (though it is getting from the `ENV`).
   
   I think this is more about a design decision, and better stay consistent across the codebase. I am not sure how other places are using this, so I think @mik-laj @potiuk @ashb can give better suggestions.




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

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



[GitHub] [airflow] mustafagok commented on pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#issuecomment-646779817


   > @mustafagok yes looks good to me. Do you want to switch some of it over to `terraform`? I already integrated it (see #8877). But we can do this in another PR.
   
   @feluelle currently I don't have much time, I prefer another PR.


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

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



[GitHub] [airflow] mik-laj commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426282931



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -51,6 +54,27 @@ def provide_aws_s3_bucket(name):
 @pytest.mark.system("amazon")
 class AmazonSystemTest(SystemTest):
 
+    @staticmethod
+    def _region_name():
+        return os.environ.get("REGION_NAME")
+
+    @staticmethod
+    def _registry_id():
+        return os.environ.get("REGISTRY_ID")
+
+    @staticmethod
+    def _repository_name():
+        return os.environ.get("REPOSITORY_NAME")
+
+    @staticmethod
+    def _execution_role_arn():
+        return os.environ.get("EXECUTION_ROLE_ARN")

Review comment:
       Personally, I would like to improve this code a bit, but until it is run on CI, I refrained. We have too many static methods that could be independent methods. This makes it difficult to write tests that apply to many cloud providers. For now, I would prefer to keep consistency in the project, because it will facilitate future refactor.




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426192142



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
       ```suggestion
           except hook.conn.exceptions.RepositoryNotFoundException:
               return None
           else:
               return response["repositories"][0]["repositoryUri"]
   ```




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r429554235



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -60,6 +84,25 @@ def execute_with_ctx(cls, cmd: List[str]):
         with provide_aws_context():
             executor.execute_cmd(cmd=cmd)
 
+    @staticmethod
+    def create_connection(aws_conn_id: str,
+                          region: str = "eu-west-1") -> None:

Review comment:
       I defined default values but not used, user have to define environment variable. You can check PR description and the docstring of ECSSystemTest class. also line 62, 80 and 113 in the airflow/tests/providers/amazon/aws/operators/test_ecs_system.py file.




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426192919



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
        The function name `is_ecr_repository_exists` is not intuitive because it does not tell that the function would also return a uri. 
   
   I suggested instead of check if a repository exists - `is_ecr_repository_exists`. You just return None if the repository not exists and name the function as `get_repository_uri`




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r427792951



##########
File path: airflow/providers/amazon/aws/example_dags/example_ecs_fargate.py
##########
@@ -64,8 +65,8 @@
     },
     network_configuration={
         "awsvpcConfiguration": {
-            "securityGroups": ["sg-123abc"],
-            "subnets": ["subnet-123456ab"],
+            "securityGroups": [os.environ.get("SECURITY_GROUP_ID")],  # or simply ["sg-123abc"],
+            "subnets": [os.environ.get("SUBNET_ID")],  # or simply ["subnet-123456ab"],

Review comment:
       You can pass fall-back values to `.get`
   ```suggestion
               "securityGroups": [os.environ.get("SECURITY_GROUP_ID", "sg-123abc")],
               "subnets": [os.environ.get("SUBNET_ID", "subnet-123456ab")],
   ```

##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,140 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AmazonSystemTest, AWS_DAG_FOLDER
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        REPOSITORY_NAME="foobar/hello-world"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b/hello-world-container"
+
+    local_image_tag = "hello"
+    local_image_context_path = "/opt/airflow/tests/providers/amazon/aws/images/ecs_test_image/"
+
+    def setUp(self):
+        print("setUp")
+        super().setUp()
+        self.create_connection(
+            aws_conn_id=self.aws_conn_id,
+            region=self._region_name(),
+        )
+
+        # create repository in ecr if it does not exist
+        repository_exists, repository_uri = self.is_ecr_repository_exists(
+            aws_conn_id=self.aws_conn_id,
+            repository_name=self._repository_name(),
+            registry_id=self._registry_id(),
+        )
+        print(f"repository_exists: {repository_exists}")
+        if not repository_exists:
+            repository_uri = self.create_ecr_repository(
+                aws_conn_id=self.aws_conn_id,
+                repository_name=self._repository_name(),
+            )
+            print(f"repository_uri: {repository_uri}")
+        full_image_name = f"{repository_uri}:latest"
+
+        # prepare (build, tag, push) image
+        self.authenticate_client_to_ecr(
+            region=self._region_name(),
+        )
+        self.build_image(
+            tag=self.local_image_tag,
+            path=self.local_image_context_path,
+        )
+        self.tag_image(
+            source=f"{self.local_image_tag}:latest",
+            target=full_image_name,
+        )
+        self.push_image(
+            tag=full_image_name,
+        )

Review comment:
       +1 for public images. 
   
   I think having the ecs system test specializing only on running the existing image from an existing ecr repo is enough to test for. Creating the image or creating the ecr repo should not be part of it (my opinion).

##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -60,6 +84,25 @@ def execute_with_ctx(cls, cmd: List[str]):
         with provide_aws_context():
             executor.execute_cmd(cmd=cmd)
 
+    @staticmethod
+    def create_connection(aws_conn_id: str,
+                          region: str = "eu-west-1") -> None:

Review comment:
       ```suggestion
                             region: str) -> None:
   ```
   Can we let the user decide that? If necessary I would add it as variable (setable via env) to the example dag.




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426192919



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
       I suggested instead of check if a repository exists - `is_ecr_repository_exists`. You just return None if the repository not exists and name the function as `get_repository_uri`




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r430553665



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")

Review comment:
       ```suggestion
   ```
   This is automatically done by the `AmazonSystemTest` class.

##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -60,6 +84,25 @@ def execute_with_ctx(cls, cmd: List[str]):
         with provide_aws_context():
             executor.execute_cmd(cmd=cmd)
 
+    @staticmethod
+    def create_connection(aws_conn_id: str,
+                          region: str = "eu-west-1") -> None:

Review comment:
       Okay, but then please remove it from here. Or is there a different reason for having this as default value?




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r429561330



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,140 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AmazonSystemTest, AWS_DAG_FOLDER
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        REPOSITORY_NAME="foobar/hello-world"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b/hello-world-container"
+
+    local_image_tag = "hello"
+    local_image_context_path = "/opt/airflow/tests/providers/amazon/aws/images/ecs_test_image/"
+
+    def setUp(self):
+        print("setUp")
+        super().setUp()
+        self.create_connection(
+            aws_conn_id=self.aws_conn_id,
+            region=self._region_name(),
+        )
+
+        # create repository in ecr if it does not exist
+        repository_exists, repository_uri = self.is_ecr_repository_exists(
+            aws_conn_id=self.aws_conn_id,
+            repository_name=self._repository_name(),
+            registry_id=self._registry_id(),
+        )
+        print(f"repository_exists: {repository_exists}")
+        if not repository_exists:
+            repository_uri = self.create_ecr_repository(
+                aws_conn_id=self.aws_conn_id,
+                repository_name=self._repository_name(),
+            )
+            print(f"repository_uri: {repository_uri}")
+        full_image_name = f"{repository_uri}:latest"
+
+        # prepare (build, tag, push) image
+        self.authenticate_client_to_ecr(
+            region=self._region_name(),
+        )
+        self.build_image(
+            tag=self.local_image_tag,
+            path=self.local_image_context_path,
+        )
+        self.tag_image(
+            source=f"{self.local_image_tag}:latest",
+            target=full_image_name,
+        )
+        self.push_image(
+            tag=full_image_name,
+        )

Review comment:
       done, thanks




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426208265



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,140 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AmazonSystemTest, AWS_DAG_FOLDER
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        REPOSITORY_NAME="foobar/hello-world"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b/hello-world-container"
+
+    local_image_tag = "hello"
+    local_image_context_path = "/opt/airflow/tests/providers/amazon/aws/images/ecs_test_image/"
+
+    def setUp(self):
+        print("setUp")
+        super().setUp()
+        self.create_connection(
+            aws_conn_id=self.aws_conn_id,
+            region=self._region_name(),
+        )
+
+        # create repository in ecr if it does not exist
+        repository_exists, repository_uri = self.is_ecr_repository_exists(
+            aws_conn_id=self.aws_conn_id,
+            repository_name=self._repository_name(),
+            registry_id=self._registry_id(),
+        )
+        print(f"repository_exists: {repository_exists}")
+        if not repository_exists:
+            repository_uri = self.create_ecr_repository(
+                aws_conn_id=self.aws_conn_id,
+                repository_name=self._repository_name(),
+            )
+            print(f"repository_uri: {repository_uri}")
+        full_image_name = f"{repository_uri}:latest"
+
+        # prepare (build, tag, push) image
+        self.authenticate_client_to_ecr(
+            region=self._region_name(),
+        )
+        self.build_image(
+            tag=self.local_image_tag,
+            path=self.local_image_context_path,
+        )
+        self.tag_image(
+            source=f"{self.local_image_tag}:latest",
+            target=full_image_name,
+        )
+        self.push_image(
+            tag=full_image_name,
+        )

Review comment:
       probably we can, I don't know, never used.
   
   we use ECS with ECR in the company I work for. It is closer example/test for `ECSOperator` :smile:
   
   building/pushing image is pretty fast when compared to AWS starting a container on ECS




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426192919



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
        The function name `is_ecr_repository_exists` is not intuitive because it does not tell that the function would also return a uri. 
   
   I suggested instead of checking if a repository exists - `is_ecr_repository_exists`. You just return None if the repository not exists and name the function as `get_ecr_repository_uri`




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r436306671



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       thank you for your review




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r429554430



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -51,6 +54,27 @@ def provide_aws_s3_bucket(name):
 @pytest.mark.system("amazon")
 class AmazonSystemTest(SystemTest):
 
+    @staticmethod
+    def _region_name():
+        return os.environ.get("REGION_NAME")
+
+    @staticmethod
+    def _registry_id():
+        return os.environ.get("REGISTRY_ID")
+
+    @staticmethod
+    def _repository_name():
+        return os.environ.get("REPOSITORY_NAME")
+
+    @staticmethod
+    def _execution_role_arn():
+        return os.environ.get("EXECUTION_ROLE_ARN")

Review comment:
       resolving this thread, reopen if you want me to change please. thank you all for review




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

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



[GitHub] [airflow] mustafagok commented on pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#issuecomment-640106679


   Necessary changes completed from reviews, Rebased, All passed :heavy_check_mark: 
   
   it is time to merge? @feluelle 


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

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



[GitHub] [airflow] feluelle commented on pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
feluelle commented on pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#issuecomment-646691513


   @mustafagok yes looks good to me. Do you want to switch some of it over to `terraform`? I already integrated it (see https://github.com/apache/airflow/pull/8877). But we can do this in another PR.


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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r436290678



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       done, can you check it please?




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r436304681



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       Nice! Thank you so much




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r432989031



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       is it `setup_class` or `setUpClass`? see [docs](https://docs.python.org/3/library/unittest.html#unittest.TestCase.setUpClass)




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r436291008



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,96 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+

Review comment:
       you are right it should work but I didn't test it with `sqlite`
   
   :heavy_check_mark: done, I have added mark-backend




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426192919



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
        The function name `is_ecr_repository_exists` is not intuitive because it does not tell that the function would also return a uri. 
   
   I suggested instead of check if a repository exists - `is_ecr_repository_exists`. You just return None if the repository not exists and name the function as `get_ecr_repository_uri`




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426193936



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,140 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AmazonSystemTest, AWS_DAG_FOLDER
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        REPOSITORY_NAME="foobar/hello-world"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b/hello-world-container"
+
+    local_image_tag = "hello"
+    local_image_context_path = "/opt/airflow/tests/providers/amazon/aws/images/ecs_test_image/"
+
+    def setUp(self):
+        print("setUp")
+        super().setUp()
+        self.create_connection(
+            aws_conn_id=self.aws_conn_id,
+            region=self._region_name(),
+        )
+
+        # create repository in ecr if it does not exist
+        repository_exists, repository_uri = self.is_ecr_repository_exists(
+            aws_conn_id=self.aws_conn_id,
+            repository_name=self._repository_name(),
+            registry_id=self._registry_id(),
+        )
+        print(f"repository_exists: {repository_exists}")
+        if not repository_exists:
+            repository_uri = self.create_ecr_repository(
+                aws_conn_id=self.aws_conn_id,
+                repository_name=self._repository_name(),
+            )
+            print(f"repository_uri: {repository_uri}")
+        full_image_name = f"{repository_uri}:latest"
+
+        # prepare (build, tag, push) image
+        self.authenticate_client_to_ecr(
+            region=self._region_name(),
+        )
+        self.build_image(
+            tag=self.local_image_tag,
+            path=self.local_image_context_path,
+        )
+        self.tag_image(
+            source=f"{self.local_image_tag}:latest",
+            target=full_image_name,
+        )
+        self.push_image(
+            tag=full_image_name,
+        )

Review comment:
       Can we use public images from DockerHub for the system test instead of building our own?




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

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



[GitHub] [airflow] feluelle merged pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
feluelle merged pull request #8888:
URL: https://github.com/apache/airflow/pull/8888


   


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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r429579522



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       
   ```suggestion
       @classmethod
       def setup_class(cli):
   ```
   Small NIT: maybe use `setup_class` is more appropriate here, in case in the future we would have more than one dags run or other task cases. Same to `teardown`.




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426209091



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -51,6 +54,27 @@ def provide_aws_s3_bucket(name):
 @pytest.mark.system("amazon")
 class AmazonSystemTest(SystemTest):
 
+    @staticmethod
+    def _region_name():
+        return os.environ.get("REGION_NAME")
+
+    @staticmethod
+    def _registry_id():
+        return os.environ.get("REGISTRY_ID")
+
+    @staticmethod
+    def _repository_name():
+        return os.environ.get("REPOSITORY_NAME")
+
+    @staticmethod
+    def _execution_role_arn():
+        return os.environ.get("EXECUTION_ROLE_ARN")

Review comment:
       `@property` is not necessarily used to return `self._x`. IMHO, `@property` fits better here because the names and functionalities are pretty much attributes access here (though it is getting from the `ENV`).
   
   I think this is more about a design decision, and better stay consistent across the codebase. I am not sure how other places are using this, so I think @mik-laj @potiuk @ashb can give better suggestions.




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r432728388



##########
File path: tests/providers/amazon/aws/operators/test_ecs_system.py
##########
@@ -0,0 +1,99 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import pytest
+
+from tests.test_utils.amazon_system_helpers import AWS_DAG_FOLDER, AmazonSystemTest
+
+
+@pytest.mark.system("amazon")
+class ECSSystemTest(AmazonSystemTest):
+    """
+    ECS System Test to run and test example ECS dags
+
+    Required variables.env file content (from your account):
+        # Auto-export all variables
+        set -a
+
+        # aws parameters
+        REGION_NAME="eu-west-1"
+        REGISTRY_ID="123456789012"
+        IMAGE="alpine:3.9"
+        SUBNET_ID="subnet-068e9654a3c357a"
+        SECURITY_GROUP_ID="sg-054dc69874a651"
+        EXECUTION_ROLE_ARN="arn:aws:iam::123456789012:role/FooBarRole"
+
+        # remove all created/existing resources flag
+        # comment out to keep resources or use empty string
+        # REMOVE_RESOURCES="True"
+    """
+
+    # should be same as in the example dag
+    aws_conn_id = "aws_ecs"
+    cluster = "c"
+    task_definition = "hello-world"
+    container = "hello-world-container"
+    awslogs_group = "/ecs/hello-world"
+    awslogs_stream_prefix = "prefix_b"  # only prefix without container name
+
+    def setUp(self):

Review comment:
       `set_up` and `tear_down` runs for every test method within the class, so if we have 3 tests (i.e. adding another 2 example dags to ECS) in the future. The `set_up` logic will run 3 times in total. 
   
   On the other hand, `setup_class`  will just apply once so all tests can reutilize the same ECS cluster.




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426207661



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -51,6 +54,27 @@ def provide_aws_s3_bucket(name):
 @pytest.mark.system("amazon")
 class AmazonSystemTest(SystemTest):
 
+    @staticmethod
+    def _region_name():
+        return os.environ.get("REGION_NAME")
+
+    @staticmethod
+    def _registry_id():
+        return os.environ.get("REGISTRY_ID")
+
+    @staticmethod
+    def _repository_name():
+        return os.environ.get("REPOSITORY_NAME")
+
+    @staticmethod
+    def _execution_role_arn():
+        return os.environ.get("EXECUTION_ROLE_ARN")

Review comment:
       not sure, these are not getter and not returning `self._x`. the `@property` decorator is used for such things afaik.
   
   also `GoogleSystemTest` class has same usage




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r429553900



##########
File path: airflow/providers/amazon/aws/example_dags/example_ecs_fargate.py
##########
@@ -64,8 +65,8 @@
     },
     network_configuration={
         "awsvpcConfiguration": {
-            "securityGroups": ["sg-123abc"],
-            "subnets": ["subnet-123456ab"],
+            "securityGroups": [os.environ.get("SECURITY_GROUP_ID")],  # or simply ["sg-123abc"],
+            "subnets": [os.environ.get("SUBNET_ID")],  # or simply ["subnet-123456ab"],

Review comment:
       these are example (placeholder) values, because this is example dag. I just want to show that the reader can use hard-coded values. Also, at the same time the example dag can be used in system tests. I can change as you suggest but the "fallback values" are not going to work.




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426207757



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -92,3 +135,310 @@ def create_emr_default_roles(cls) -> None:
         """
         cmd = ["aws", "emr", "create-default-roles"]
         cls.execute_with_ctx(cmd)
+
+    @staticmethod
+    def create_ecr_repository(aws_conn_id: str,
+                              repository_name: str) -> str:
+        """
+        Create repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to create in aws ecr
+        :type repository_name: str
+        :return: uri of the created repository
+            uri format: `registry_id`.dkr.ecr.`region`.amazonaws.com/`repository_name`
+        :rtype: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        response = hook.conn.create_repository(
+            repositoryName=repository_name,
+            imageTagMutability="MUTABLE",
+            imageScanningConfiguration={
+                "scanOnPush": False
+            },
+        )
+        return response["repository"]["repositoryUri"]
+
+    @staticmethod
+    def delete_ecr_repository(aws_conn_id: str,
+                              repository_name: str,
+                              registry_id: str) -> None:
+        """
+        Delete repository in ecr with given name
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to delete in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+        hook.conn.delete_repository(
+            registryId=registry_id,
+            repositoryName=repository_name,
+            force=True,  # also delete images
+        )
+
+    @staticmethod
+    def is_ecr_repository_exists(aws_conn_id: str,
+                                 repository_name: str,
+                                 registry_id: str) -> Tuple[bool, str]:
+        """
+        Check whether given repository exists in ecr
+
+        :param aws_conn_id: id of the aws connection to use when creating boto3 client/resource
+        :type aws_conn_id: str
+        :param repository_name: name of the repository to check in aws ecr
+        :type repository_name: str
+        :param registry_id: aws account id associated with the registry that contains the repository to delete
+        :type registry_id: str
+        :return: True and uri of the found repository if given repository exists in ecr,
+            else False and empty string
+        :rtype: Tuple[bool, str]
+        """
+        hook = AwsBaseHook(
+            aws_conn_id=aws_conn_id,
+            client_type="ecr",
+        )
+
+        try:
+            response = hook.conn.describe_repositories(
+                registryId=registry_id,
+                repositoryNames=[repository_name],
+            )
+        except hook.conn.exceptions.RepositoryNotFoundException:
+            # return False if repository does not exist
+            return False, ""
+
+        # return True if repository exists
+        return True, response["repositories"][0]["repositoryUri"]

Review comment:
       good point, thank you. I will change it.
   
   at first it was just returning True or False, then I added uri :sweat_smile: 




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

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



[GitHub] [airflow] mustafagok commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r432721671



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -60,6 +84,25 @@ def execute_with_ctx(cls, cmd: List[str]):
         with provide_aws_context():
             executor.execute_cmd(cmd=cmd)
 
+    @staticmethod
+    def create_connection(aws_conn_id: str,
+                          region: str = "eu-west-1") -> None:

Review comment:
       done :heavy_check_mark:  thanks




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #8888: Add AWS ECS system test

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #8888:
URL: https://github.com/apache/airflow/pull/8888#discussion_r426193317



##########
File path: tests/test_utils/amazon_system_helpers.py
##########
@@ -51,6 +54,27 @@ def provide_aws_s3_bucket(name):
 @pytest.mark.system("amazon")
 class AmazonSystemTest(SystemTest):
 
+    @staticmethod
+    def _region_name():
+        return os.environ.get("REGION_NAME")
+
+    @staticmethod
+    def _registry_id():
+        return os.environ.get("REGISTRY_ID")
+
+    @staticmethod
+    def _repository_name():
+        return os.environ.get("REPOSITORY_NAME")
+
+    @staticmethod
+    def _execution_role_arn():
+        return os.environ.get("EXECUTION_ROLE_ARN")

Review comment:
       ```suggestion
       @property
       def _region_name():
           return os.environ.get("REGION_NAME")
   
       @property
       def _registry_id():
           return os.environ.get("REGISTRY_ID")
   
       @property
       def _repository_name():
           return os.environ.get("REPOSITORY_NAME")
   
       @property
       def _execution_role_arn():
           return os.environ.get("EXECUTION_ROLE_ARN")
   ```




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

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