Skip to content

Commit 0f19a93

Browse files
authored
Remove GKEStartPodOperator when backporting (#7908)
1 parent daad60b commit 0f19a93

File tree

4 files changed

+93
-2
lines changed

4 files changed

+93
-2
lines changed
Lines changed: 59 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,59 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one
3+
# or more contributor license agreements. See the NOTICE file
4+
# distributed with this work for additional information
5+
# regarding copyright ownership. The ASF licenses this file
6+
# to you under the Apache License, Version 2.0 (the
7+
# "License"); you may not use this file except in compliance
8+
# with the License. You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing,
13+
# software distributed under the License is distributed on an
14+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
# KIND, either express or implied. See the License for the
16+
# specific language governing permissions and limitations
17+
# under the License.
18+
"""
19+
Example Airflow DAG for Google Kubernetes Engine.
20+
"""
21+
22+
import os
23+
24+
from airflow import models
25+
from airflow.providers.google.cloud.operators.kubernetes_engine import (
26+
GKECreateClusterOperator, GKEDeleteClusterOperator,
27+
)
28+
from airflow.utils.dates import days_ago
29+
30+
GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project")
31+
GCP_LOCATION = os.environ.get("GCP_GKE_LOCATION", "europe-north1-a")
32+
CLUSTER_NAME = os.environ.get("GCP_GKE_CLUSTER_NAME", "cluster-name")
33+
34+
CLUSTER = {"name": CLUSTER_NAME, "initial_node_count": 1}
35+
36+
default_args = {"start_date": days_ago(1)}
37+
38+
39+
with models.DAG(
40+
"example_gcp_gke_setup",
41+
default_args=default_args,
42+
schedule_interval=None, # Override to match your needs
43+
tags=['example'],
44+
) as dag:
45+
create_cluster = GKECreateClusterOperator(
46+
task_id="create_cluster",
47+
project_id=GCP_PROJECT_ID,
48+
location=GCP_LOCATION,
49+
body=CLUSTER,
50+
)
51+
52+
delete_cluster = GKEDeleteClusterOperator(
53+
task_id="delete_cluster",
54+
name=CLUSTER_NAME,
55+
project_id=GCP_PROJECT_ID,
56+
location=GCP_LOCATION,
57+
)
58+
59+
create_cluster >> delete_cluster

β€Žbackport_packages/setup_backport_packages.py

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -170,6 +170,21 @@ def add_provide_context_to_python_operator(node: LN, capture: Capture, filename:
170170
provide_context_arg.prefix = fn_args.children[0].prefix
171171
fn_args.append_child(provide_context_arg)
172172

173+
def remove_class(qry, class_name) -> None:
174+
def _remover(node: LN, capture: Capture, filename: Filename) -> None:
175+
if node.type == 300:
176+
for ch in node.post_order():
177+
if isinstance(ch, Leaf) and ch.value == class_name:
178+
if ch.next_sibling and ch.next_sibling.value == ",":
179+
ch.next_sibling.remove()
180+
ch.remove()
181+
elif node.type == 311:
182+
node.parent.remove()
183+
else:
184+
node.remove()
185+
186+
qry.select_class(class_name).modify(_remover)
187+
173188
changes = [
174189
("airflow.operators.bash", "airflow.operators.bash_operator"),
175190
("airflow.operators.python", "airflow.operators.python_operator"),
@@ -228,6 +243,8 @@ def add_provide_context_to_python_operator(node: LN, capture: Capture, filename:
228243
.modify(add_provide_context_to_python_operator)
229244
)
230245

246+
remove_class(qry, "GKEStartPodOperator")
247+
231248
qry.execute(write=True, silent=False, interactive=False)
232249

233250

β€Žtests/conftest.py

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -178,11 +178,14 @@ def pytest_configure(config):
178178
"markers", "system(name): mark test to run with named system"
179179
)
180180
config.addinivalue_line(
181-
"markers", "long_running(name): mark test that run for a long time (many minutes)"
181+
"markers", "long_running: mark test that run for a long time (many minutes)"
182182
)
183183
config.addinivalue_line(
184184
"markers", "credential_file(name): mark tests that require credential file in CREDENTIALS_DIR"
185185
)
186+
config.addinivalue_line(
187+
"markers", "airflow_2: mark tests that works only on Airflow 2.0 / master"
188+
)
186189

187190

188191
def skip_if_not_marked_with_integration(selected_integrations, item):
@@ -292,6 +295,12 @@ def skip_if_credential_file_missing(item):
292295
format(path=credential_path, item=item))
293296

294297

298+
def skip_if_airflow_2_test(item):
299+
for _ in item.iter_markers(name="airflow_2"):
300+
if os.environ.get("RUN_AIRFLOW_1_10") == "true":
301+
pytest.skip("The test works only with Airflow 2.0 / master branch")
302+
303+
295304
def pytest_runtest_setup(item):
296305
selected_integrations = item.config.getoption("--integrations")
297306
selected_integrations_list = selected_integrations.split(",") if selected_integrations else []
@@ -319,3 +328,4 @@ def pytest_runtest_setup(item):
319328
if not include_long_running:
320329
skip_long_running_test(item)
321330
skip_if_credential_file_missing(item)
331+
skip_if_airflow_2_test(item)

β€Žtests/providers/google/cloud/operators/test_kubernetes_engine_system.py

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,11 @@
2424
@pytest.mark.backend("mysql", "postgres")
2525
@pytest.mark.credential_file(GCP_GKE_KEY)
2626
class KubernetesEngineExampleDagTest(GoogleSystemTest):
27+
@pytest.mark.airflow_2
2728
@provide_gcp_context(GCP_GKE_KEY)
28-
def test_run_example_dag(self):
29+
def test_run_example_gcp_gke(self):
2930
self.run_dag('example_gcp_gke', CLOUD_DAG_FOLDER)
31+
32+
@provide_gcp_context(GCP_GKE_KEY)
33+
def test_run_example_gcp_gke_setup(self):
34+
self.run_dag('example_gcp_gke_setup', CLOUD_DAG_FOLDER)

0 commit comments

Comments
 (0)