Skip to content

Commit

Permalink
Add AWS Fargate profile support (#18645)
Browse files Browse the repository at this point in the history
  • Loading branch information
ferruzzi authored Oct 8, 2021
1 parent 9344c34 commit ea8f478
Show file tree
Hide file tree
Showing 15 changed files with 1,693 additions and 145 deletions.
16 changes: 4 additions & 12 deletions airflow/providers/amazon/aws/example_dags/example_eks_templated.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from datetime import datetime

from airflow.models.dag import DAG
from airflow.providers.amazon.aws.hooks.eks import ClusterStates, NodegroupStates
Expand All @@ -25,7 +26,6 @@
EKSPodOperator,
)
from airflow.providers.amazon.aws.sensors.eks import EKSClusterStateSensor, EKSNodegroupStateSensor
from airflow.utils.dates import days_ago

# Example Jinja Template format, substitute your values:
"""
Expand All @@ -45,8 +45,9 @@

with DAG(
dag_id='to-publish-manuals-templated',
default_args={'cluster_name': "{{ dag_run.conf['cluster_name'] }}"},
schedule_interval=None,
start_date=days_ago(2),
start_date=datetime(2021, 1, 1),
max_active_runs=1,
tags=['example', 'templated'],
# render_template_as_native_obj=True is what converts the Jinja to Python objects, instead of a string.
Expand All @@ -57,36 +58,31 @@
create_cluster = EKSCreateClusterOperator(
task_id='create_eks_cluster',
compute=None,
cluster_name="{{ dag_run.conf['cluster_name'] }}",
cluster_role_arn="{{ dag_run.conf['cluster_role_arn'] }}",
resources_vpc_config="{{ dag_run.conf['resources_vpc_config'] }}",
)

await_create_cluster = EKSClusterStateSensor(
task_id='wait_for_create_cluster',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
target_state=ClusterStates.ACTIVE,
)

create_nodegroup = EKSCreateNodegroupOperator(
task_id='create_eks_nodegroup',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
nodegroup_name="{{ dag_run.conf['nodegroup_name'] }}",
nodegroup_subnets="{{ dag_run.conf['nodegroup_subnets'] }}",
nodegroup_role_arn="{{ dag_run.conf['nodegroup_role_arn'] }}",
)

await_create_nodegroup = EKSNodegroupStateSensor(
task_id='wait_for_create_nodegroup',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
nodegroup_name="{{ dag_run.conf['nodegroup_name'] }}",
target_state=NodegroupStates.ACTIVE,
)

start_pod = EKSPodOperator(
task_id="run_pod",
pod_name="start_pod",
cluster_name="{{ dag_run.conf['cluster_name'] }}",
pod_name="run_pod",
image="amazon/aws-cli:latest",
cmds=["sh", "-c", "ls"],
labels={"demo": "hello_world"},
Expand All @@ -97,25 +93,21 @@

delete_nodegroup = EKSDeleteNodegroupOperator(
task_id='delete_eks_nodegroup',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
nodegroup_name="{{ dag_run.conf['nodegroup_name'] }}",
)

await_delete_nodegroup = EKSNodegroupStateSensor(
task_id='wait_for_delete_nodegroup',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
nodegroup_name="{{ dag_run.conf['nodegroup_name'] }}",
target_state=NodegroupStates.NONEXISTENT,
)

delete_cluster = EKSDeleteClusterOperator(
task_id='delete_eks_cluster',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
)

await_delete_cluster = EKSClusterStateSensor(
task_id='wait_for_delete_cluster',
cluster_name="{{ dag_run.conf['cluster_name'] }}",
target_state=ClusterStates.NONEXISTENT,
)

Expand Down
Original file line number Diff line number Diff line change
@@ -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.
from datetime import datetime
from os import environ

from airflow.models.dag import DAG
from airflow.providers.amazon.aws.hooks.eks import ClusterStates, FargateProfileStates
from airflow.providers.amazon.aws.operators.eks import (
EKSCreateClusterOperator,
EKSDeleteClusterOperator,
EKSPodOperator,
)
from airflow.providers.amazon.aws.sensors.eks import EKSClusterStateSensor, EKSFargateProfileStateSensor

CLUSTER_NAME = 'fargate-all-in-one'
FARGATE_PROFILE_NAME = f'{CLUSTER_NAME}-profile'

ROLE_ARN = environ.get('EKS_DEMO_ROLE_ARN', 'arn:aws:iam::123456789012:role/role_name')
SUBNETS = environ.get('EKS_DEMO_SUBNETS', 'subnet-12345ab subnet-67890cd').split(' ')
VPC_CONFIG = {
'subnetIds': SUBNETS,
'endpointPublicAccess': True,
'endpointPrivateAccess': False,
}


with DAG(
dag_id='example-create-cluster-and-fargate-all-in-one',
default_args={'cluster_name': CLUSTER_NAME},
schedule_interval=None,
start_date=datetime(2021, 1, 1),
max_active_runs=1,
tags=['example'],
) as dag:

# [START howto_operator_eks_create_cluster_with_fargate_profile]
# Create an Amazon EKS cluster control plane and an AWS Fargate compute platform in one step.
create_cluster_and_fargate_profile = EKSCreateClusterOperator(
task_id='create_eks_cluster_and_fargate_profile',
cluster_role_arn=ROLE_ARN,
resources_vpc_config=VPC_CONFIG,
compute='fargate',
fargate_profile_name=FARGATE_PROFILE_NAME,
# Opting to use the same ARN for the cluster and the pod here,
# but a different ARN could be configured and passed if desired.
fargate_pod_execution_role_arn=ROLE_ARN,
)
# [END howto_operator_eks_create_cluster_with_fargate_profile]

await_create_fargate_profile = EKSFargateProfileStateSensor(
task_id='wait_for_create_fargate_profile',
fargate_profile_name=FARGATE_PROFILE_NAME,
target_state=FargateProfileStates.ACTIVE,
)

start_pod = EKSPodOperator(
task_id="run_pod",
pod_name="run_pod",
image="amazon/aws-cli:latest",
cmds=["sh", "-c", "echo Test Airflow; date"],
labels={"demo": "hello_world"},
get_logs=True,
# Delete the pod when it reaches its final state, or the execution is interrupted.
is_delete_operator_pod=True,
)

# An Amazon EKS cluster can not be deleted with attached resources such as nodegroups or Fargate profiles.
# Setting the `force` to `True` will delete any attached resources before deleting the cluster.
delete_all = EKSDeleteClusterOperator(
task_id='delete_fargate_profile_and_cluster', force_delete_compute=True
)

await_delete_cluster = EKSClusterStateSensor(
task_id='wait_for_delete_cluster',
target_state=ClusterStates.NONEXISTENT,
)

(
create_cluster_and_fargate_profile
>> await_create_fargate_profile
>> start_pod
>> delete_all
>> await_delete_cluster
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
# 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 datetime import datetime
from os import environ

from airflow.models.dag import DAG
from airflow.providers.amazon.aws.hooks.eks import ClusterStates, FargateProfileStates
from airflow.providers.amazon.aws.operators.eks import (
EKSCreateClusterOperator,
EKSCreateFargateProfileOperator,
EKSDeleteClusterOperator,
EKSDeleteFargateProfileOperator,
EKSPodOperator,
)
from airflow.providers.amazon.aws.sensors.eks import EKSClusterStateSensor, EKSFargateProfileStateSensor

CLUSTER_NAME = 'fargate-demo'
FARGATE_PROFILE_NAME = f'{CLUSTER_NAME}-profile'
SELECTORS = environ.get('FARGATE_SELECTORS', [{'namespace': 'default'}])

ROLE_ARN = environ.get('EKS_DEMO_ROLE_ARN', 'arn:aws:iam::123456789012:role/role_name')
SUBNETS = environ.get('EKS_DEMO_SUBNETS', 'subnet-12345ab subnet-67890cd').split(' ')
VPC_CONFIG = {
'subnetIds': SUBNETS,
'endpointPublicAccess': True,
'endpointPrivateAccess': False,
}


with DAG(
dag_id='example_eks_with_fargate_profile_dag',
default_args={'cluster_name': CLUSTER_NAME},
schedule_interval=None,
start_date=datetime(2021, 1, 1),
max_active_runs=1,
tags=['example'],
) as dag:

# Create an Amazon EKS Cluster control plane without attaching a compute service.
create_cluster = EKSCreateClusterOperator(
task_id='create_eks_cluster',
cluster_role_arn=ROLE_ARN,
resources_vpc_config=VPC_CONFIG,
compute=None,
)

await_create_cluster = EKSClusterStateSensor(
task_id='wait_for_create_cluster',
target_state=ClusterStates.ACTIVE,
)

# [START howto_operator_eks_create_fargate_profile]
create_fargate_profile = EKSCreateFargateProfileOperator(
task_id='create_eks_fargate_profile',
pod_execution_role_arn=ROLE_ARN,
fargate_profile_name=FARGATE_PROFILE_NAME,
selectors=SELECTORS,
)
# [END howto_operator_eks_create_fargate_profile]

await_create_fargate_profile = EKSFargateProfileStateSensor(
task_id='wait_for_create_fargate_profile',
fargate_profile_name=FARGATE_PROFILE_NAME,
target_state=FargateProfileStates.ACTIVE,
)

start_pod = EKSPodOperator(
task_id="run_pod",
pod_name="run_pod",
image="amazon/aws-cli:latest",
cmds=["sh", "-c", "echo Test Airflow; date"],
labels={"demo": "hello_world"},
get_logs=True,
# Delete the pod when it reaches its final state, or the execution is interrupted.
is_delete_operator_pod=True,
)

# [START howto_operator_eks_delete_fargate_profile]
delete_fargate_profile = EKSDeleteFargateProfileOperator(
task_id='delete_eks_fargate_profile',
fargate_profile_name=FARGATE_PROFILE_NAME,
)
# [END howto_operator_eks_delete_fargate_profile]

await_delete_fargate_profile = EKSFargateProfileStateSensor(
task_id='wait_for_delete_fargate_profile',
fargate_profile_name=FARGATE_PROFILE_NAME,
target_state=FargateProfileStates.NONEXISTENT,
)

delete_cluster = EKSDeleteClusterOperator(task_id='delete_eks_cluster')

await_delete_cluster = EKSClusterStateSensor(
task_id='wait_for_delete_cluster',
target_state=ClusterStates.NONEXISTENT,
)

(
create_cluster
>> await_create_cluster
>> create_fargate_profile
>> await_create_fargate_profile
>> start_pod
>> delete_fargate_profile
>> await_delete_fargate_profile
>> delete_cluster
>> await_delete_cluster
)
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from datetime import datetime
from os import environ

from airflow.models.dag import DAG
Expand All @@ -24,7 +25,6 @@
EKSPodOperator,
)
from airflow.providers.amazon.aws.sensors.eks import EKSClusterStateSensor, EKSNodegroupStateSensor
from airflow.utils.dates import days_ago

CLUSTER_NAME = environ.get('EKS_CLUSTER_NAME', 'eks-demo')
NODEGROUP_NAME = f'{CLUSTER_NAME}-nodegroup'
Expand All @@ -39,8 +39,9 @@

with DAG(
dag_id='example_eks_using_defaults_dag',
default_args={'cluster_name': CLUSTER_NAME},
schedule_interval=None,
start_date=days_ago(2),
start_date=datetime(2021, 1, 1),
max_active_runs=1,
tags=['example'],
) as dag:
Expand All @@ -49,7 +50,6 @@
# Create an Amazon EKS cluster control plane and an EKS nodegroup compute platform in one step.
create_cluster_and_nodegroup = EKSCreateClusterOperator(
task_id='create_eks_cluster_and_nodegroup',
cluster_name=CLUSTER_NAME,
nodegroup_name=NODEGROUP_NAME,
cluster_role_arn=ROLE_ARN,
nodegroup_role_arn=ROLE_ARN,
Expand All @@ -63,15 +63,13 @@

await_create_nodegroup = EKSNodegroupStateSensor(
task_id='wait_for_create_nodegroup',
cluster_name=CLUSTER_NAME,
nodegroup_name=NODEGROUP_NAME,
target_state=NodegroupStates.ACTIVE,
)

start_pod = EKSPodOperator(
task_id="run_pod",
pod_name="run_pod",
cluster_name=CLUSTER_NAME,
image="amazon/aws-cli:latest",
cmds=["sh", "-c", "echo Test Airflow; date"],
labels={"demo": "hello_world"},
Expand All @@ -81,16 +79,13 @@
)

# [START howto_operator_eks_force_delete_cluster]
# An Amazon EKS cluster can not be deleted with attached resources.
# An Amazon EKS cluster can not be deleted with attached resources such as nodegroups or Fargate profiles.
# Setting the `force` to `True` will delete any attached resources before deleting the cluster.
delete_all = EKSDeleteClusterOperator(
task_id='delete_nodegroup_and_cluster', cluster_name=CLUSTER_NAME, force_delete_compute=True
)
delete_all = EKSDeleteClusterOperator(task_id='delete_nodegroup_and_cluster', force_delete_compute=True)
# [END howto_operator_eks_force_delete_cluster]

await_delete_cluster = EKSClusterStateSensor(
task_id='wait_for_delete_cluster',
cluster_name=CLUSTER_NAME,
target_state=ClusterStates.NONEXISTENT,
)

Expand Down
Loading

0 comments on commit ea8f478

Please sign in to comment.