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 2022/06/06 08:31:50 UTC

[GitHub] [airflow] Taragolis opened a new pull request, #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Taragolis opened a new pull request, #24246:
URL: https://github.com/apache/airflow/pull/24246

   related: #24198
   
   Try to make amazon-provider operators (and later sensors) more consistent.
   At that moment I try to use Generic and additional MixIn class for passing parameters to hooks and simplify operators itself
   
   Affected different types of operators
   - airflow.airflow.providers.amazon.aws.operators.athena.AthenaOperator
   - airflow.airflow.providers.amazon.aws.operators.aws_lambda.AwsLambdaInvokeFunctionOperator
   - airflow.airflow.providers.amazon.aws.operators.batch.BatchOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksCreateClusterOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksCreateNodegroupOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksCreateFargateProfileOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksDeleteClusterOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksDeleteNodegroupOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksDeleteFargateProfileOperator
   - airflow.airflow.providers.amazon.aws.operators.eks.EksPodOperator


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

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

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


[GitHub] [airflow] Taragolis commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r892148384


##########
airflow/providers/amazon/aws/operators/base.py:
##########
@@ -0,0 +1,97 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property

Review Comment:
   BTW, since new provider versions expected min Airflow 2.2 could we replace this by
   
   ```python
   from airflow.compat.functools import cached_property
   ```



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

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

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


[GitHub] [airflow] ferruzzi commented on pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on PR #24246:
URL: https://github.com/apache/airflow/pull/24246#issuecomment-1148087628

   @o-nikolas should have eyes on this one.   It implements a couple of things we've discussed.


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

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

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890688056


##########
airflow/providers/amazon/aws/operators/batch.py:
##########
@@ -95,6 +95,9 @@ class BatchOperator(BaseOperator):
     )
     template_fields_renderers = {"overrides": "json", "parameters": "json"}
 
+    aws_hook_class = BatchClientHook
+    aws_hook_class_fields = set(AwsBaseOperator.aws_hook_class_fields) | {"max_retries", "status_retries"}

Review Comment:
   Easier to do
   
   ```suggestion
       aws_hook_class_fields = {
           "max_retries",
           "status_retries",
           *AwsBaseOperator.aws_hook_class_fields,
       }
   ```



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

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

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


[GitHub] [airflow] Taragolis commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890954255


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }
+
+    def __init__(
+        self,
+        *,
+        aws_conn_id: Optional[str] = "aws_default",
+        region_name: Optional[str] = None,
+        config: Optional[Config] = None,
+        **kwargs,
+    ) -> None:
+        self.aws_conn_id = aws_conn_id
+
+        region = kwargs.pop("region", None)
+        if region:
+            warnings.warn(
+                'Parameter `region` is deprecated. Please use `region_name` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if region_name:
+                raise AirflowException("Either `region_name` or `region` can be provided, not both.")
+            region_name = region
+
+        self.region_name = region_name
+        self.config = config
+
+        # Remove hook fields from keywords arguments
+        for kw in self.aws_hook_class_fields:
+            if kw in kwargs:
+                if not hasattr(self, kw):
+                    setattr(self, kw, kwargs.pop(kw, None))
+                else:
+                    kwargs.pop(kw, None)
+
+        super().__init__(**kwargs)
+
+    @cached_property
+    def hook(self) -> AwsHookClass:
+        """Create and return an AthenaHook."""

Review Comment:
   Whoops copy-paste issue



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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r891503879


##########
airflow/providers/amazon/aws/operators/eks.py:
##########
@@ -151,9 +153,7 @@ def __init__(
         self.fargate_pod_execution_role_arn = fargate_pod_execution_role_arn
         self.fargate_selectors = fargate_selectors or [{"namespace": DEFAULT_NAMESPACE_NAME}]
         self.create_fargate_profile_kwargs = create_fargate_profile_kwargs or {}
-        self.aws_conn_id = aws_conn_id
-        self.region = region
-        super().__init__(**kwargs)
+        super().__init__(aws_conn_id=aws_conn_id, region=region, **kwargs)

Review Comment:
   Nevermind, I guess it's a safe hedge against the operator pulling them out.



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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r891321668


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }

Review Comment:
   That seems to make sense to me. :+1: 



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

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

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


[GitHub] [airflow] Taragolis commented on pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #24246:
URL: https://github.com/apache/airflow/pull/24246#issuecomment-1150103823

   Recent changes:
   - `aws_hook_class_kwargs` class argument is optional
   - check `AwsBaseHook.conn_name_attr` for non-default `aws_conn_id` and use it in hook
   - for EKS operator replace `region` to `region_name`. Previously created DAGs shouldn't failed
   - Add tests
   
   cc: @ferruzzi @uranusjr @o-nikolas 


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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890685833


##########
airflow/providers/amazon/aws/operators/eks.py:
##########
@@ -151,9 +153,7 @@ def __init__(
         self.fargate_pod_execution_role_arn = fargate_pod_execution_role_arn
         self.fargate_selectors = fargate_selectors or [{"namespace": DEFAULT_NAMESPACE_NAME}]
         self.create_fargate_profile_kwargs = create_fargate_profile_kwargs or {}
-        self.aws_conn_id = aws_conn_id
-        self.region = region
-        super().__init__(**kwargs)
+        super().__init__(aws_conn_id=aws_conn_id, region=region, **kwargs)

Review Comment:
   Is this right?  Here and a couple of times below:  You have 'region_name' in the templatable fields, then pass `region` here.



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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890685833


##########
airflow/providers/amazon/aws/operators/eks.py:
##########
@@ -151,9 +153,7 @@ def __init__(
         self.fargate_pod_execution_role_arn = fargate_pod_execution_role_arn
         self.fargate_selectors = fargate_selectors or [{"namespace": DEFAULT_NAMESPACE_NAME}]
         self.create_fargate_profile_kwargs = create_fargate_profile_kwargs or {}
-        self.aws_conn_id = aws_conn_id
-        self.region = region
-        super().__init__(**kwargs)
+        super().__init__(aws_conn_id=aws_conn_id, region=region, **kwargs)

Review Comment:
   Is this right?  You have 'region_name' in the templatable fields, then pass `region` here.



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

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

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890690008


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }

Review Comment:
   I think we only do if the checker cannot infer (as is the case for `template_fields`). But if you want to explicitly declare, use `Set[str]` instead.



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

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

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


[GitHub] [airflow] Taragolis commented on pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #24246:
URL: https://github.com/apache/airflow/pull/24246#issuecomment-1147304218

   looks on failed tests 👀 


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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r891319491


##########
airflow/providers/amazon/aws/operators/eks.py:
##########
@@ -151,9 +153,7 @@ def __init__(
         self.fargate_pod_execution_role_arn = fargate_pod_execution_role_arn
         self.fargate_selectors = fargate_selectors or [{"namespace": DEFAULT_NAMESPACE_NAME}]
         self.create_fargate_profile_kwargs = create_fargate_profile_kwargs or {}
-        self.aws_conn_id = aws_conn_id
-        self.region = region
-        super().__init__(**kwargs)
+        super().__init__(aws_conn_id=aws_conn_id, region=region, **kwargs)

Review Comment:
   Also, is there a reason to call out `aws_conn_id` and `region_name` in the new L156?  Why not just drop lines 154 and 155 and leave 156 as `super().__init__(**kwargs)`.  If they haven't been pulled out of kwargs, it's the same thing, right?



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

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

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


[GitHub] [airflow] Taragolis commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r892787703


##########
airflow/providers/amazon/aws/operators/base.py:
##########
@@ -0,0 +1,97 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   makes sense



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

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

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


[GitHub] [airflow] github-actions[bot] commented on pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #24246:
URL: https://github.com/apache/airflow/pull/24246#issuecomment-1193425980

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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

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

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890686542


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }

Review Comment:
   ```suggestion
       aws_hook_class_fields = {
           "aws_conn_id",
           "region_name",
           "config",
       }
   ```
   
   The type checker is smart enough to infer the type from the right-hand value automatically.



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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890687610


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }

Review Comment:
   I thought we were still supposed to explicitly declare them as well. 



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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r892775749


##########
airflow/providers/amazon/aws/operators/base.py:
##########
@@ -0,0 +1,97 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Just noticed the filename, can you change it to base_aws.py to match the equivalent hook?



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

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

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


[GitHub] [airflow] Taragolis closed pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis closed pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent
URL: https://github.com/apache/airflow/pull/24246


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

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

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


[GitHub] [airflow] Taragolis commented on pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #24246:
URL: https://github.com/apache/airflow/pull/24246#issuecomment-1193841630

   Close for now, plan back to this  PR after changes in Hook(s) 


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

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

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


[GitHub] [airflow] ferruzzi commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890681643


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }
+
+    def __init__(
+        self,
+        *,
+        aws_conn_id: Optional[str] = "aws_default",
+        region_name: Optional[str] = None,
+        config: Optional[Config] = None,
+        **kwargs,
+    ) -> None:
+        self.aws_conn_id = aws_conn_id
+
+        region = kwargs.pop("region", None)
+        if region:
+            warnings.warn(
+                'Parameter `region` is deprecated. Please use `region_name` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if region_name:
+                raise AirflowException("Either `region_name` or `region` can be provided, not both.")
+            region_name = region
+
+        self.region_name = region_name
+        self.config = config
+
+        # Remove hook fields from keywords arguments
+        for kw in self.aws_hook_class_fields:
+            if kw in kwargs:
+                if not hasattr(self, kw):
+                    setattr(self, kw, kwargs.pop(kw, None))
+                else:
+                    kwargs.pop(kw, None)
+
+        super().__init__(**kwargs)
+
+    @cached_property
+    def hook(self) -> AwsHookClass:
+        """Create and return an AthenaHook."""

Review Comment:
   ```suggestion
           """Create and return an AWS Hook."""
   ```



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

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

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


[GitHub] [airflow] ferruzzi commented on pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on PR #24246:
URL: https://github.com/apache/airflow/pull/24246#issuecomment-1148097166

   > 
   
   You just need to add the base operator module to `airflow/providers/amazon/provider.yaml` around L217


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

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

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


[GitHub] [airflow] Taragolis commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r890961901


##########
airflow/providers/amazon/aws/operators/eks.py:
##########
@@ -151,9 +153,7 @@ def __init__(
         self.fargate_pod_execution_role_arn = fargate_pod_execution_role_arn
         self.fargate_selectors = fargate_selectors or [{"namespace": DEFAULT_NAMESPACE_NAME}]
         self.create_fargate_profile_kwargs = create_fargate_profile_kwargs or {}
-        self.aws_conn_id = aws_conn_id
-        self.region = region
-        super().__init__(**kwargs)
+        super().__init__(aws_conn_id=aws_conn_id, region=region, **kwargs)

Review Comment:
   For EKS operators `region` use as argument operators I've keep this argument right now for compatibility just need update docstring
   
   This argument will set to `region_name` class instance attribute, so it should be fine with templated fields
   https://github.com/apache/airflow/blob/60b8227dba97d81bc898f7cf3bf13c9bbfafaa2e/airflow/providers/amazon/aws/operators/aws_base.py#L63-L74



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

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

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


[GitHub] [airflow] Taragolis commented on a diff in pull request #24246: Make amazon athena, batch, lambda, and eks operators more consistent

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #24246:
URL: https://github.com/apache/airflow/pull/24246#discussion_r891002672


##########
airflow/providers/amazon/aws/operators/aws_base.py:
##########
@@ -0,0 +1,91 @@
+# 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 sys
+import warnings
+from typing import Generic, Optional, Set, Type, TypeVar
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+from botocore.config import Config
+
+from airflow import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+AwsHookClass = TypeVar("AwsHookClass", bound=AwsBaseHook)
+
+
+class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
+    """Base implementations for amazon-provider operators.
+
+    :param aws_conn_id: aws connection to use
+    :param region_name: (optional) region name to use in AWS Hook.
+        Override the region_name in connection (if provided)
+    :param config: Configuration for botocore client.
+        (https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html)
+    """
+
+    aws_hook_class: Type[AwsHookClass]
+    aws_hook_class_fields: Set = {
+        "aws_conn_id",
+        "region_name",
+        "config",
+    }

Review Comment:
   BTW, I just think about how it would be done better. 
   
   Every hook based on `AwsBaseHook` could use `aws_conn_id`, `region_name`, `config`. That mean we do not need explicit set this arguments as part of attribute `aws_hook_class_fields` and only set for additional arguments
   
   And second part probably `config` is to broad name for operator attribute, and I think better use `botocore_config` or `boto3_config` instead
   
   ```python
   class AwsBaseOperator(BaseOperator, Generic[AwsHookClass]):
       aws_hook_class: Type[AwsHookClass]
       aws_hook_class_fields: Optional[Set[str]] = None
   
       ...
   
       @cached_property
       def hook(self) -> AwsHookClass:
           """Create and return an AWS Hook."""
           kwargs = {kw: getattr(self, kw) for kw in self.aws_hook_class_fields}
           return self.aws_hook_class(
               aws_conn_id=self.aws_conn_id,
               region_name=self.region_name,
               config=self.botocore_config,
               **kwargs
           )
   ```
   
   WDYT @ferruzzi @uranusjr ?



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

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

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