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/03/15 15:17:22 UTC

[GitHub] [airflow] mustafagok opened a new pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

mustafagok opened a new pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731
 
 
   Create EC2 Hook, Operator and Sensor
   
   - New hook to interact with AWS EC2 Service.
   - New operator to manage AWS EC2 instance and to change instance state by applying given operation using boto3.
   - New sensor to check the state of the AWS EC2 instance until state of the instance become equal to the target state.
   
   ---
   Issue link: WILL BE INSERTED BY [boring-cyborg](https://github.com/kaxil/boring-cyborg)
   
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Commit message/PR title starts with `[AIRFLOW-NNNN]`. AIRFLOW-NNNN = JIRA ID<sup>*</sup>
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [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).
   
   <sup>*</sup> For document-only changes commit message can start with `[AIRFLOW-XXXX]`.
   
   ---
   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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392913117
 
 

 ##########
 File path: airflow/providers/amazon/aws/sensors/ec2.py
 ##########
 @@ -0,0 +1,78 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Sensor(BaseSensorOperator):
+    """
+    Check the state of the AWS EC2 instance until
+    state of the instance become equal to the target state.
+
+    :param target_state: target state of instances
+    :type target_state: str
+    :param instance_id: id of the AWS EC2 Instances
+    :type instance_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    """
+
+    template_fields = ["target_state", "region_name"]
+    ui_color = "#cc8811"
+    ui_fgcolor = "#ffffff"
+    valid_states = ["running", "stopped"]
 
 Review comment:
   added, thx

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392703975
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
+        super().__init__(*args, **kwargs)
+        self.operation = operation
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+        self.target_state = self.operation_target_state_map[self.operation]
+        self.hook = self.get_hook()
+
+    def execute(self, context):
+        self.log.info("Executing: %s %s", self.operation, self.instance_id)
+
+        instance = self.hook.get_conn().Instance(id=self.instance_id)
+
+        if self.operation == "start":
 
 Review comment:
   It is recommended to create one operator for each operation.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-599234519
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=h1) Report
   > Merging [#7731](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0740dafa0a971b9dc05c6a1b3f034b869c827aa8&el=desc) will **decrease** coverage by `0.25%`.
   > The diff coverage is `95.94%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7731/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7731      +/-   ##
   ==========================================
   - Coverage   86.93%   86.67%   -0.26%     
   ==========================================
     Files         915      918       +3     
     Lines       44158    44232      +74     
   ==========================================
   - Hits        38387    38339      -48     
   - Misses       5771     5893     +122     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/amazon/aws/operators/ec2.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9vcGVyYXRvcnMvZWMyLnB5) | `91.89% <91.89%> (ø)` | |
   | [airflow/providers/amazon/aws/hooks/ec2.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9ob29rcy9lYzIucHk=) | `100.00% <100.00%> (ø)` | |
   | [airflow/providers/amazon/aws/sensors/ec2.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9zZW5zb3JzL2VjMi5weQ==) | `100.00% <100.00%> (ø)` | |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0.00%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0.00%> (-47.06%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0.00%> (-45.08%)` | :arrow_down: |
   | [...viders/cncf/kubernetes/operators/kubernetes\_pod.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvY25jZi9rdWJlcm5ldGVzL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZC5weQ==) | `69.69% <0.00%> (-25.26%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0.00%> (-23.53%)` | :arrow_down: |
   | ... and [1 more](https://codecov.io/gh/apache/airflow/pull/7731/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=footer). Last update [0740daf...5772f7a](https://codecov.io/gh/apache/airflow/pull/7731?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392914054
 
 

 ##########
 File path: airflow/providers/amazon/aws/sensors/ec2.py
 ##########
 @@ -0,0 +1,78 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Sensor(BaseSensorOperator):
+    """
+    Check the state of the AWS EC2 instance until
+    state of the instance become equal to the target state.
+
+    :param target_state: target state of instances
+    :type target_state: str
+    :param instance_id: id of the AWS EC2 Instances
+    :type instance_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    """
+
+    template_fields = ["target_state", "region_name"]
+    ui_color = "#cc8811"
+    ui_fgcolor = "#ffffff"
+    valid_states = ["running", "stopped"]
+
+    @apply_defaults
+    def __init__(self,
+                 target_state: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        if target_state not in self.valid_states:
+            raise AirflowException(f"Invalid target_state: {target_state}")
+        super().__init__(*args, **kwargs)
+        self.target_state = target_state
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.hook = self.get_hook()
 
 Review comment:
   I didn't know that, fixed, thank you. 
   P.S.: There are some bad practices in other operators.

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-607469395
 
 
   > I've fixed up the other ticket, so we just need to give that a final review+merge
   
   @ashb please let me know when #7541 merged. I will make necessary changes for EC2Hook. Thank you :slightly_smiling_face: 

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392704096
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,48 @@
+#
+# 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 typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+    """
+
+    def __init__(self,
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        self.region_name = region_name
+        self.conn = None
+        super().__init__(*args, **kwargs)
+
+    def get_conn(self):
+        """
+        Return self.conn, if it is None initialize ec2 resource object.
+
+        :return: ec2 resource
+        :rtype: boto3.resource
+        """
+        # set self.conn in first call
+        if not self.conn:
+            self.conn = self.get_resource_type("ec2", self.region_name)
+        return self.conn
 
 Review comment:
   This operator has too few methods. In an ideal world, operators should never call the get_conn method directly.

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


With regards,
Apache Git Services

[GitHub] [airflow] nuclearpinguin commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
nuclearpinguin commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392975245
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2_start_instance.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2StartInstanceOperator(BaseOperator):
+    """
+    Start AWS EC2 instance using boto3.
+
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+
+    @apply_defaults
+    def __init__(self,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        super().__init__(*args, **kwargs)
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+
+    def execute(self, context):
+        ec2_hook = self.get_hook()
+        self.log.info("Starting EC2 instance %s", self.instance_id)
+
+        instance = ec2_hook.get_instance(instance_id=self.instance_id)
+        instance.start()
+        instance_state = ec2_hook.get_instance_state(
+            instance_id=self.instance_id
+        )
+
+        while instance_state != "running":
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(self.check_interval)
+            instance_state = ec2_hook.get_instance_state(
+                instance_id=self.instance_id
+            )
 
 Review comment:
   This method seems to be used in both operators. Can we extract it to hook as something like `wait_for_status` ?

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r395983309
 
 

 ##########
 File path: docs/operators-and-hooks-ref.rst
 ##########
 @@ -360,12 +360,19 @@ These integrations allow you to perform various operations within the Amazon Web
      -
      -
 
-   * - `Amazon EC2 <https://aws.amazon.com/ec2/>`__
+   * - `Amazon ECS <https://aws.amazon.com/ecs/>`__
      -
      -
      - :mod:`airflow.providers.amazon.aws.operators.ecs`
      -
 
+   * - `Amazon EC2 <https://aws.amazon.com/ec2/>`__
+     -
+     - :mod:`airflow.providers.amazon.aws.hooks.ec2`
+     - :mod:`airflow.providers.amazon.aws.operators.ec2_start_instance`,
+       :mod:`airflow.providers.amazon.aws.operators.ec2_stop_instance`,
+     - :mod:`airflow.providers.amazon.aws.sensors.ec2_instance_state`
+
 
 Review comment:
   Can you put this block before `Amazon ECS`, 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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r396005805
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,98 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+    """
+
+    def __init__(self,
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        self.region_name = region_name
+        self.conn = None
+        super().__init__(*args, **kwargs)
+
+    def get_conn(self):
+        """
+        Return self.conn, if it is None initialize ec2 resource object.
+
+        :return: ec2 resource
+        :rtype: boto3.resource
+        """
+        # set self.conn in first call
+        if not self.conn:
+            self.conn = self.get_resource_type("ec2", self.region_name)
+        return self.conn
+
+    def get_instance(self, instance_id: str):
+        """
+        Get EC2 instance by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: Instance object
+        :rtype: ec2.Instance
+        """
+        return self.get_conn().Instance(id=instance_id)
+
+    def get_instance_state(self, instance_id: str) -> str:
+        """
+        Get EC2 instance state by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: current state of the instance
+        :rtype: str
+        """
+        return self.get_instance(instance_id=instance_id).state["Name"]
+
+    def wait_for_state(self,
+                       instance_id: str,
+                       target_state: str,
+                       check_interval: float) -> None:
+        """
+        Wait EC2 instance until its state is equal to the target_state.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :param target_state: target state of instance
+        :type target_state: str
+        :param check_interval: time in seconds that the job should wait in
+            between each instance state checks until operation is completed
+        :type check_interval: float
+        :return: None
+        :rtype: None
+        """
+        instance_state = self.get_instance_state(
+            instance_id=instance_id
+        )
+        while instance_state != target_state:
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(check_interval)
+            instance_state = self.get_instance_state(
+                instance_id=instance_id
+            )
 
 Review comment:
   I think both options are fine (in this case) but my pref would be the second. You can keep yours. LGTM too :)

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-610827401
 
 
   @mustafagok Right, that other PR has been merged (sorry for the slow down) - could you rebase on the latest master and update appropriately 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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392883812
 
 

 ##########
 File path: airflow/providers/amazon/aws/sensors/ec2.py
 ##########
 @@ -0,0 +1,78 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Sensor(BaseSensorOperator):
+    """
+    Check the state of the AWS EC2 instance until
+    state of the instance become equal to the target state.
+
+    :param target_state: target state of instances
+    :type target_state: str
+    :param instance_id: id of the AWS EC2 Instances
+    :type instance_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    """
+
+    template_fields = ["target_state", "region_name"]
+    ui_color = "#cc8811"
+    ui_fgcolor = "#ffffff"
+    valid_states = ["running", "stopped"]
+
+    @apply_defaults
+    def __init__(self,
+                 target_state: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        if target_state not in self.valid_states:
+            raise AirflowException(f"Invalid target_state: {target_state}")
+        super().__init__(*args, **kwargs)
+        self.target_state = target_state
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.hook = self.get_hook()
 
 Review comment:
   The other problem with calling this here: the region_name template _will not_ have been rendered yet. That only happens just before execute is called.

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


With regards,
Apache Git Services

[GitHub] [airflow] nuclearpinguin commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
nuclearpinguin commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392975840
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2_start_instance.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2StartInstanceOperator(BaseOperator):
+    """
+    Start AWS EC2 instance using boto3.
+
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+
+    @apply_defaults
+    def __init__(self,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        super().__init__(*args, **kwargs)
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+
+    def execute(self, context):
+        ec2_hook = self.get_hook()
+        self.log.info("Starting EC2 instance %s", self.instance_id)
+
+        instance = ec2_hook.get_instance(instance_id=self.instance_id)
+        instance.start()
+        instance_state = ec2_hook.get_instance_state(
+            instance_id=self.instance_id
+        )
+
+        while instance_state != "running":
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(self.check_interval)
+            instance_state = ec2_hook.get_instance_state(
+                instance_id=self.instance_id
+            )
+
+    def get_hook(self):
+        """
+        Return EC2Hook object.
+
+        :return: ec2 hook
+        :rtype: EC2Hook
+        """
+        return EC2Hook(
+            aws_conn_id=self.aws_conn_id,
+            region_name=self.region_name
+        )
 
 Review comment:
   Personally I would remove this method and just done:
   ```python
   hook = EC2Hook(
                aws_conn_id=self.aws_conn_id,
                region_name=self.region_name
            )
   ```
   in `execute` method :)

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-599506603
 
 
   See also https://github.com/apache/airflow/pull/7541 (which I'm in the process of fixing up) - it shouldn't be a big change for you to rebase on to that once I have it passing/merged.

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392979402
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2_start_instance.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2StartInstanceOperator(BaseOperator):
+    """
+    Start AWS EC2 instance using boto3.
+
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+
+    @apply_defaults
+    def __init__(self,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        super().__init__(*args, **kwargs)
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+
+    def execute(self, context):
+        ec2_hook = self.get_hook()
+        self.log.info("Starting EC2 instance %s", self.instance_id)
+
+        instance = ec2_hook.get_instance(instance_id=self.instance_id)
+        instance.start()
+        instance_state = ec2_hook.get_instance_state(
+            instance_id=self.instance_id
+        )
+
+        while instance_state != "running":
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(self.check_interval)
+            instance_state = ec2_hook.get_instance_state(
+                instance_id=self.instance_id
+            )
+
+    def get_hook(self):
+        """
+        Return EC2Hook object.
+
+        :return: ec2 hook
+        :rtype: EC2Hook
+        """
+        return EC2Hook(
+            aws_conn_id=self.aws_conn_id,
+            region_name=self.region_name
+        )
 
 Review comment:
   You're right, a little unnecessary. I saw examples like this and thought "I should write like this". I will change 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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392914383
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
+        super().__init__(*args, **kwargs)
+        self.operation = operation
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+        self.target_state = self.operation_target_state_map[self.operation]
+        self.hook = self.get_hook()
 
 Review comment:
   fixed

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-607095732
 
 
   I've fixed up the other ticket, so we just need to give that a final review+merge

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392914708
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
 
 Review comment:
   This part removed, after spliting EC2StartInstanceOperator and EC2StopInstanceOperator

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392705410
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,48 @@
+#
+# 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 typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+    """
+
+    def __init__(self,
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        self.region_name = region_name
+        self.conn = None
+        super().__init__(*args, **kwargs)
+
+    def get_conn(self):
+        """
+        Return self.conn, if it is None initialize ec2 resource object.
+
+        :return: ec2 resource
+        :rtype: boto3.resource
+        """
+        # set self.conn in first call
+        if not self.conn:
+            self.conn = self.get_resource_type("ec2", self.region_name)
+        return self.conn
 
 Review comment:
   You are right, I think I should add a method named get_instance(instance_id) or get_instance_state(instance_id) to thi EC2Hook class. Then I can call this from operator/sensor. What do you think?

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392705107
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
+        super().__init__(*args, **kwargs)
+        self.operation = operation
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+        self.target_state = self.operation_target_state_map[self.operation]
+        self.hook = self.get_hook()
+
+    def execute(self, context):
+        self.log.info("Executing: %s %s", self.operation, self.instance_id)
+
+        instance = self.hook.get_conn().Instance(id=self.instance_id)
+
+        if self.operation == "start":
 
 Review comment:
   I can write EC2StartInstanceOperator and EC2StopInstanceOperator, but these operators will be identical except 2 lines. What do you suggest?

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392912965
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
 
 Review comment:
   I have split it into 2 operators: EC2StartInstanceOperator and EC2StopInstanceOperator

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r396003763
 
 

 ##########
 File path: docs/operators-and-hooks-ref.rst
 ##########
 @@ -360,12 +360,19 @@ These integrations allow you to perform various operations within the Amazon Web
      -
      -
 
-   * - `Amazon EC2 <https://aws.amazon.com/ec2/>`__
+   * - `Amazon ECS <https://aws.amazon.com/ecs/>`__
      -
      -
      - :mod:`airflow.providers.amazon.aws.operators.ecs`
      -
 
+   * - `Amazon EC2 <https://aws.amazon.com/ec2/>`__
+     -
+     - :mod:`airflow.providers.amazon.aws.hooks.ec2`
+     - :mod:`airflow.providers.amazon.aws.operators.ec2_start_instance`,
+       :mod:`airflow.providers.amazon.aws.operators.ec2_stop_instance`,
+     - :mod:`airflow.providers.amazon.aws.sensors.ec2_instance_state`
+
 
 Review comment:
   Sure, I did not noticed that they are alphabetical

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb edited a comment on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-610827401
 
 
   @mustafagok Right, that other PR has been merged (sorry for the slow down) - could you rebase on the latest master and update appropriately please? Oh you are way ahead of me.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392882215
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
+        super().__init__(*args, **kwargs)
+        self.operation = operation
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+        self.target_state = self.operation_target_state_map[self.operation]
+        self.hook = self.get_hook()
 
 Review comment:
   Do not create hooks in Operator constructors -- this could open a network connection before the task is parsed (i.e. at DAG parse time).

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392705410
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,48 @@
+#
+# 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 typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+    """
+
+    def __init__(self,
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        self.region_name = region_name
+        self.conn = None
+        super().__init__(*args, **kwargs)
+
+    def get_conn(self):
+        """
+        Return self.conn, if it is None initialize ec2 resource object.
+
+        :return: ec2 resource
+        :rtype: boto3.resource
+        """
+        # set self.conn in first call
+        if not self.conn:
+            self.conn = self.get_resource_type("ec2", self.region_name)
+        return self.conn
 
 Review comment:
   You are right, I think I should add a method named get_instance(instance_id) or get_instance_state(instance_id) to this EC2Hook class. Then I can call this from operator/sensor. What do you think?

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


With regards,
Apache Git Services

[GitHub] [airflow] nuclearpinguin commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
nuclearpinguin commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392713098
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
+        super().__init__(*args, **kwargs)
+        self.operation = operation
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+        self.target_state = self.operation_target_state_map[self.operation]
+        self.hook = self.get_hook()
+
+    def execute(self, context):
+        self.log.info("Executing: %s %s", self.operation, self.instance_id)
+
+        instance = self.hook.get_conn().Instance(id=self.instance_id)
+
+        if self.operation == "start":
 
 Review comment:
   I agree with @mik-laj, we should two operators even if the difference is small :)

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r405351466
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# 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 time
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self,
+                 *args,
+                 **kwargs):
+        super().__init__(resource_type="ec2", *args, **kwargs)
+
+    def get_instance(self, instance_id: str):
+        """
+        Get EC2 instance by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: Instance object
+        :rtype: ec2.Instance
+        """
+        return self.get_conn().Instance(id=instance_id)
 
 Review comment:
   ```suggestion
           return self.conn.Instance(id=instance_id)
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392883476
 
 

 ##########
 File path: airflow/providers/amazon/aws/sensors/ec2.py
 ##########
 @@ -0,0 +1,78 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Sensor(BaseSensorOperator):
+    """
+    Check the state of the AWS EC2 instance until
+    state of the instance become equal to the target state.
+
+    :param target_state: target state of instances
+    :type target_state: str
+    :param instance_id: id of the AWS EC2 Instances
+    :type instance_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    """
+
+    template_fields = ["target_state", "region_name"]
+    ui_color = "#cc8811"
+    ui_fgcolor = "#ffffff"
+    valid_states = ["running", "stopped"]
+
+    @apply_defaults
+    def __init__(self,
+                 target_state: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        if target_state not in self.valid_states:
+            raise AirflowException(f"Invalid target_state: {target_state}")
+        super().__init__(*args, **kwargs)
+        self.target_state = target_state
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.hook = self.get_hook()
 
 Review comment:
   Same 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392882983
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
 
 Review comment:
   ```suggestion
   class EC2InstanceOperator(BaseOperator):
   ```
   
   There are more than just Instances in EC2 

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392883328
 
 

 ##########
 File path: airflow/providers/amazon/aws/sensors/ec2.py
 ##########
 @@ -0,0 +1,78 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Sensor(BaseSensorOperator):
+    """
+    Check the state of the AWS EC2 instance until
+    state of the instance become equal to the target state.
+
+    :param target_state: target state of instances
+    :type target_state: str
+    :param instance_id: id of the AWS EC2 Instances
+    :type instance_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    """
+
+    template_fields = ["target_state", "region_name"]
+    ui_color = "#cc8811"
+    ui_fgcolor = "#ffffff"
+    valid_states = ["running", "stopped"]
 
 Review comment:
   What about terminated?

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r395983169
 
 

 ##########
 File path: airflow/providers/amazon/aws/sensors/ec2_instance_state.py
 ##########
 @@ -0,0 +1,77 @@
+#
+# 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 typing import Optional
+
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.sensors.base_sensor_operator import BaseSensorOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2InstanceStateSensor(BaseSensorOperator):
+    """
+    Check the state of the AWS EC2 instance until
+    state of the instance become equal to the target state.
+
+    :param target_state: target state of instance
+    :type target_state: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    """
+
+    template_fields = ["target_state", "region_name"]
+    ui_color = "#cc8811"
+    ui_fgcolor = "#ffffff"
+    valid_states = ["running", "stopped", "terminated"]
+
+    @apply_defaults
+    def __init__(self,
+                 target_state: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        if target_state not in self.valid_states:
+            raise ValueError(f"Invalid target_state: {target_state}")
+        super().__init__(*args, **kwargs)
+        self.target_state = target_state
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+
+    def poke(self, context):
+        ec2_hook = self.get_hook()
+        instance = ec2_hook.get_instance(instance_id=self.instance_id)
+        self.log.info("instance state: %s", instance.state)
+        return instance.state["Name"] == self.target_state
 
 Review comment:
   ```suggestion
           instance_state = ec2_hook.get_instance_state(instance_id=self.instance_id)
           self.log.info("instance state: %s", instance_state)
           return instance_state  == self.target_state
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-599771515
 
 
   > See also #7541 (which I'm in the process of fixing up) - it shouldn't be a big change for you to rebase on to that once I have it passing/merged.
   
   No it isn't a big change and I understand what to do, but that PR haven't been merged yet. And mine is ready I think. What are you doing in this situations?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-600081644
 
 
   > No it isn't a big change and I understand what to do, but that PR haven't been merged yet. And mine is ready I think. What are you doing in this situations?
   
   We often join forces - choose one (more advanced)  change, the other person reviews and comments/suggests. Then the first person adds co-authored-by https://help.github.com/en/github/committing-changes-to-your-project/creating-a-commit-with-multiple-authors and submit only one change with two authors.

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
turbaszek commented on issue #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#issuecomment-607051121
 
 
   > See also #7541 (which I'm in the process of fixing up) - it shouldn't be a big change for you to rebase on to that once I have it passing/merged.
   
   @ashb should we merge or wait?

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392884669
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2.py
 ##########
 @@ -0,0 +1,102 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2Operator(BaseOperator):
+    """
+    Manage AWS EC2 instance using boto3.
+    Change instance state by applying given operation.
+
+    :param operation: action to be taken on AWS EC2 instance
+        valid values: "start", "stop"
+    :type operation: str
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["operation", "region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+    valid_operations = ["start", "stop"]
+    operation_target_state_map = {
+        "start": "running",
+        "stop": "stopped",
+    }
+
+    @apply_defaults
+    def __init__(self,
+                 operation: str,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        if operation not in self.valid_operations:
+            raise AirflowException(f"Invalid operation: {operation}")
 
 Review comment:
   ```suggestion
               raise ValueError(f"Invalid operation: {operation}")
   ```
   
   > Raised when an operation or function receives an argument that has the right type but an inappropriate value, and the situation is not described by a more precise exception such as IndexError.

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r392977905
 
 

 ##########
 File path: airflow/providers/amazon/aws/operators/ec2_start_instance.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook
+from airflow.utils.decorators import apply_defaults
+
+
+class EC2StartInstanceOperator(BaseOperator):
+    """
+    Start AWS EC2 instance using boto3.
+
+    :param instance_id: id of the AWS EC2 instance
+    :type instance_id: str
+    :param aws_conn_id: aws connection to use
+    :type aws_conn_id: str
+    :param region_name: (optional) aws region name associated with the client
+    :type region_name: Optional[str]
+    :param check_interval: time in seconds that the job should wait in
+        between each instance state checks until operation is completed
+    :type check_interval: float
+    """
+
+    template_fields = ["region_name"]
+    ui_color = "#eeaa11"
+    ui_fgcolor = "#ffffff"
+
+    @apply_defaults
+    def __init__(self,
+                 instance_id: str,
+                 aws_conn_id: str = "aws_default",
+                 region_name: Optional[str] = None,
+                 check_interval: float = 15,
+                 *args,
+                 **kwargs):
+        super().__init__(*args, **kwargs)
+        self.instance_id = instance_id
+        self.aws_conn_id = aws_conn_id
+        self.region_name = region_name
+        self.check_interval = check_interval
+
+    def execute(self, context):
+        ec2_hook = self.get_hook()
+        self.log.info("Starting EC2 instance %s", self.instance_id)
+
+        instance = ec2_hook.get_instance(instance_id=self.instance_id)
+        instance.start()
+        instance_state = ec2_hook.get_instance_state(
+            instance_id=self.instance_id
+        )
+
+        while instance_state != "running":
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(self.check_interval)
+            instance_state = ec2_hook.get_instance_state(
+                instance_id=self.instance_id
+            )
 
 Review comment:
   Sure, I will.

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r405791980
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# 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 time
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self,
+                 *args,
+                 **kwargs):
+        super().__init__(resource_type="ec2", *args, **kwargs)
+
+    def get_instance(self, instance_id: str):
+        """
+        Get EC2 instance by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: Instance object
+        :rtype: ec2.Instance
+        """
+        return self.get_conn().Instance(id=instance_id)
 
 Review comment:
   Oh, I have changed all other get_conn() methods with self.conn, but forgot this one :)
   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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r395982917
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,98 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+    """
+
+    def __init__(self,
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        self.region_name = region_name
+        self.conn = None
+        super().__init__(*args, **kwargs)
+
+    def get_conn(self):
+        """
+        Return self.conn, if it is None initialize ec2 resource object.
+
+        :return: ec2 resource
+        :rtype: boto3.resource
+        """
+        # set self.conn in first call
+        if not self.conn:
+            self.conn = self.get_resource_type("ec2", self.region_name)
+        return self.conn
+
+    def get_instance(self, instance_id: str):
+        """
+        Get EC2 instance by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: Instance object
+        :rtype: ec2.Instance
+        """
+        return self.get_conn().Instance(id=instance_id)
+
+    def get_instance_state(self, instance_id: str) -> str:
+        """
+        Get EC2 instance state by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: current state of the instance
+        :rtype: str
+        """
+        return self.get_instance(instance_id=instance_id).state["Name"]
+
+    def wait_for_state(self,
+                       instance_id: str,
+                       target_state: str,
+                       check_interval: float) -> None:
+        """
+        Wait EC2 instance until its state is equal to the target_state.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :param target_state: target state of instance
+        :type target_state: str
+        :param check_interval: time in seconds that the job should wait in
+            between each instance state checks until operation is completed
+        :type check_interval: float
+        :return: None
+        :rtype: None
+        """
+        instance_state = self.get_instance_state(
+            instance_id=instance_id
+        )
+        while instance_state != target_state:
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(check_interval)
+            instance_state = self.get_instance_state(
+                instance_id=instance_id
+            )
 
 Review comment:
   ```suggestion
           while True:
               instance_state = self.get_instance_state(
                   instance_id=instance_id
               )
               self.log.info("instance state: %s", instance_state)
               if instance_state == target_state: 
                   break
               time.sleep(check_interval)
   ```
   WDYT?

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


With regards,
Apache Git Services

[GitHub] [airflow] mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor

Posted by GitBox <gi...@apache.org>.
mustafagok commented on a change in pull request #7731: [AIRFLOW-7068] Create EC2 Hook, Operator and Sensor
URL: https://github.com/apache/airflow/pull/7731#discussion_r396004322
 
 

 ##########
 File path: airflow/providers/amazon/aws/hooks/ec2.py
 ##########
 @@ -0,0 +1,98 @@
+#
+# 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 time
+from typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class EC2Hook(AwsBaseHook):
+    """
+    Interact with AWS EC2 Service.
+    """
+
+    def __init__(self,
+                 region_name: Optional[str] = None,
+                 *args,
+                 **kwargs):
+        self.region_name = region_name
+        self.conn = None
+        super().__init__(*args, **kwargs)
+
+    def get_conn(self):
+        """
+        Return self.conn, if it is None initialize ec2 resource object.
+
+        :return: ec2 resource
+        :rtype: boto3.resource
+        """
+        # set self.conn in first call
+        if not self.conn:
+            self.conn = self.get_resource_type("ec2", self.region_name)
+        return self.conn
+
+    def get_instance(self, instance_id: str):
+        """
+        Get EC2 instance by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: Instance object
+        :rtype: ec2.Instance
+        """
+        return self.get_conn().Instance(id=instance_id)
+
+    def get_instance_state(self, instance_id: str) -> str:
+        """
+        Get EC2 instance state by id and return it.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :return: current state of the instance
+        :rtype: str
+        """
+        return self.get_instance(instance_id=instance_id).state["Name"]
+
+    def wait_for_state(self,
+                       instance_id: str,
+                       target_state: str,
+                       check_interval: float) -> None:
+        """
+        Wait EC2 instance until its state is equal to the target_state.
+
+        :param instance_id: id of the AWS EC2 instance
+        :type instance_id: str
+        :param target_state: target state of instance
+        :type target_state: str
+        :param check_interval: time in seconds that the job should wait in
+            between each instance state checks until operation is completed
+        :type check_interval: float
+        :return: None
+        :rtype: None
+        """
+        instance_state = self.get_instance_state(
+            instance_id=instance_id
+        )
+        while instance_state != target_state:
+            self.log.info("instance state: %s", instance_state)
+            time.sleep(check_interval)
+            instance_state = self.get_instance_state(
+                instance_id=instance_id
+            )
 
 Review comment:
   Not sure, I heard that while True is a bad practice since it's hard to follow the loop condition, but the other one has some code duplication. So, I am not sure, what do you think about this, do you know any post related to this situation. I am facing this situation frequently and still not know what to do :(
   [example question frrom stackoverflow](https://stackoverflow.com/questions/743164/emulate-a-do-while-loop-in-python)

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


With regards,
Apache Git Services