apache-airflow-providers-amazon 8.26.0rc2__py3-none-any.whl → 8.27.0rc1__py3-none-any.whl
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- airflow/providers/amazon/__init__.py +1 -1
- airflow/providers/amazon/aws/datasets/__init__.py +16 -0
- airflow/providers/amazon/aws/datasets/s3.py +45 -0
- airflow/providers/amazon/aws/executors/batch/batch_executor.py +20 -13
- airflow/providers/amazon/aws/executors/ecs/ecs_executor.py +24 -13
- airflow/providers/amazon/aws/hooks/kinesis_analytics.py +65 -0
- airflow/providers/amazon/aws/hooks/rds.py +3 -3
- airflow/providers/amazon/aws/hooks/s3.py +26 -1
- airflow/providers/amazon/aws/hooks/step_function.py +18 -0
- airflow/providers/amazon/aws/operators/athena.py +16 -17
- airflow/providers/amazon/aws/operators/kinesis_analytics.py +348 -0
- airflow/providers/amazon/aws/operators/rds.py +17 -20
- airflow/providers/amazon/aws/operators/redshift_cluster.py +4 -2
- airflow/providers/amazon/aws/operators/s3.py +7 -11
- airflow/providers/amazon/aws/operators/sagemaker.py +2 -3
- airflow/providers/amazon/aws/operators/step_function.py +12 -2
- airflow/providers/amazon/aws/sensors/kinesis_analytics.py +234 -0
- airflow/providers/amazon/aws/transfers/redshift_to_s3.py +1 -0
- airflow/providers/amazon/aws/transfers/s3_to_redshift.py +1 -0
- airflow/providers/amazon/aws/triggers/emr.py +3 -1
- airflow/providers/amazon/aws/triggers/kinesis_analytics.py +69 -0
- airflow/providers/amazon/aws/waiters/kinesisanalyticsv2.json +151 -0
- airflow/providers/amazon/aws/waiters/rds.json +253 -0
- airflow/providers/amazon/get_provider_info.py +35 -2
- {apache_airflow_providers_amazon-8.26.0rc2.dist-info → apache_airflow_providers_amazon-8.27.0rc1.dist-info}/METADATA +29 -22
- {apache_airflow_providers_amazon-8.26.0rc2.dist-info → apache_airflow_providers_amazon-8.27.0rc1.dist-info}/RECORD +28 -20
- {apache_airflow_providers_amazon-8.26.0rc2.dist-info → apache_airflow_providers_amazon-8.27.0rc1.dist-info}/WHEEL +0 -0
- {apache_airflow_providers_amazon-8.26.0rc2.dist-info → apache_airflow_providers_amazon-8.27.0rc1.dist-info}/entry_points.txt +0 -0
@@ -0,0 +1,348 @@
|
|
1
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
2
|
+
# or more contributor license agreements. See the NOTICE file
|
3
|
+
# distributed with this work for additional information
|
4
|
+
# regarding copyright ownership. The ASF licenses this file
|
5
|
+
# to you under the Apache License, Version 2.0 (the
|
6
|
+
# "License"); you may not use this file except in compliance
|
7
|
+
# with the License. You may obtain a copy of the License at
|
8
|
+
#
|
9
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
10
|
+
#
|
11
|
+
# Unless required by applicable law or agreed to in writing,
|
12
|
+
# software distributed under the License is distributed on an
|
13
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
14
|
+
# KIND, either express or implied. See the License for the
|
15
|
+
# specific language governing permissions and limitations
|
16
|
+
# under the License.
|
17
|
+
from __future__ import annotations
|
18
|
+
|
19
|
+
from typing import TYPE_CHECKING, Any, Sequence
|
20
|
+
|
21
|
+
from botocore.exceptions import ClientError
|
22
|
+
|
23
|
+
from airflow.configuration import conf
|
24
|
+
from airflow.exceptions import AirflowException
|
25
|
+
from airflow.providers.amazon.aws.hooks.kinesis_analytics import KinesisAnalyticsV2Hook
|
26
|
+
from airflow.providers.amazon.aws.operators.base_aws import AwsBaseOperator
|
27
|
+
from airflow.providers.amazon.aws.triggers.kinesis_analytics import (
|
28
|
+
KinesisAnalyticsV2ApplicationOperationCompleteTrigger,
|
29
|
+
)
|
30
|
+
from airflow.providers.amazon.aws.utils import validate_execute_complete_event
|
31
|
+
from airflow.providers.amazon.aws.utils.mixins import aws_template_fields
|
32
|
+
|
33
|
+
if TYPE_CHECKING:
|
34
|
+
from airflow.utils.context import Context
|
35
|
+
|
36
|
+
|
37
|
+
class KinesisAnalyticsV2CreateApplicationOperator(AwsBaseOperator[KinesisAnalyticsV2Hook]):
|
38
|
+
"""
|
39
|
+
Creates an AWS Managed Service for Apache Flink application.
|
40
|
+
|
41
|
+
.. seealso::
|
42
|
+
For more information on how to use this operator, take a look at the guide:
|
43
|
+
:ref:`howto/operator:KinesisAnalyticsV2CreateApplicationOperator`
|
44
|
+
|
45
|
+
:param application_name: The name of application. (templated)
|
46
|
+
:param runtime_environment: The runtime environment for the application. (templated)
|
47
|
+
:param service_execution_role: The IAM role used by the application to access services. (templated)
|
48
|
+
:param create_application_kwargs: Create application extra properties. (templated)
|
49
|
+
:param application_description: A summary description of the application. (templated)
|
50
|
+
|
51
|
+
:param aws_conn_id: The Airflow connection used for AWS credentials.
|
52
|
+
If this is ``None`` or empty then the default boto3 behaviour is used. If
|
53
|
+
running Airflow in a distributed manner and aws_conn_id is None or
|
54
|
+
empty, then default boto3 configuration would be used (and must be
|
55
|
+
maintained on each worker node).
|
56
|
+
:param region_name: AWS region_name. If not specified then the default boto3 behaviour is used.
|
57
|
+
:param verify: Whether to verify SSL certificates. See:
|
58
|
+
https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html
|
59
|
+
:param botocore_config: Configuration dictionary (key-values) for botocore client. See:
|
60
|
+
https://botocore.amazonaws.com/v1/documentation/api/latest/reference/config.html
|
61
|
+
"""
|
62
|
+
|
63
|
+
aws_hook_class = KinesisAnalyticsV2Hook
|
64
|
+
ui_color = "#44b5e2"
|
65
|
+
|
66
|
+
template_fields: Sequence[str] = aws_template_fields(
|
67
|
+
"application_name",
|
68
|
+
"runtime_environment",
|
69
|
+
"service_execution_role",
|
70
|
+
"create_application_kwargs",
|
71
|
+
"application_description",
|
72
|
+
)
|
73
|
+
template_fields_renderers: dict = {
|
74
|
+
"create_application_kwargs": "json",
|
75
|
+
}
|
76
|
+
|
77
|
+
def __init__(
|
78
|
+
self,
|
79
|
+
application_name: str,
|
80
|
+
runtime_environment: str,
|
81
|
+
service_execution_role: str,
|
82
|
+
create_application_kwargs: dict[str, Any] | None = None,
|
83
|
+
application_description: str = "Managed Service for Apache Flink application created from Airflow",
|
84
|
+
**kwargs,
|
85
|
+
):
|
86
|
+
super().__init__(**kwargs)
|
87
|
+
self.application_name = application_name
|
88
|
+
self.runtime_environment = runtime_environment
|
89
|
+
self.service_execution_role = service_execution_role
|
90
|
+
self.create_application_kwargs = create_application_kwargs or {}
|
91
|
+
self.application_description = application_description
|
92
|
+
|
93
|
+
def execute(self, context: Context) -> dict[str, str]:
|
94
|
+
self.log.info("Creating AWS Managed Service for Apache Flink application %s.", self.application_name)
|
95
|
+
try:
|
96
|
+
response = self.hook.conn.create_application(
|
97
|
+
ApplicationName=self.application_name,
|
98
|
+
ApplicationDescription=self.application_description,
|
99
|
+
RuntimeEnvironment=self.runtime_environment,
|
100
|
+
ServiceExecutionRole=self.service_execution_role,
|
101
|
+
**self.create_application_kwargs,
|
102
|
+
)
|
103
|
+
except ClientError as error:
|
104
|
+
raise AirflowException(
|
105
|
+
f"AWS Managed Service for Apache Flink application creation failed: {error.response['Error']['Message']}"
|
106
|
+
)
|
107
|
+
|
108
|
+
self.log.info(
|
109
|
+
"AWS Managed Service for Apache Flink application created successfully %s.",
|
110
|
+
self.application_name,
|
111
|
+
)
|
112
|
+
|
113
|
+
return {"ApplicationARN": response["ApplicationDetail"]["ApplicationARN"]}
|
114
|
+
|
115
|
+
|
116
|
+
class KinesisAnalyticsV2StartApplicationOperator(AwsBaseOperator[KinesisAnalyticsV2Hook]):
|
117
|
+
"""
|
118
|
+
Starts an AWS Managed Service for Apache Flink application.
|
119
|
+
|
120
|
+
.. seealso::
|
121
|
+
For more information on how to use this operator, take a look at the guide:
|
122
|
+
:ref:`howto/operator:KinesisAnalyticsV2StartApplicationOperator`
|
123
|
+
|
124
|
+
:param application_name: The name of application. (templated)
|
125
|
+
:param run_configuration: Application properties to start Apache Flink Job. (templated)
|
126
|
+
|
127
|
+
:param wait_for_completion: Whether to wait for job to stop. (default: True)
|
128
|
+
:param waiter_delay: Time in seconds to wait between status checks. (default: 60)
|
129
|
+
:param waiter_max_attempts: Maximum number of attempts to check for job completion. (default: 20)
|
130
|
+
:param deferrable: If True, the operator will wait asynchronously for the job to stop.
|
131
|
+
This implies waiting for completion. This mode requires aiobotocore module to be installed.
|
132
|
+
(default: False)
|
133
|
+
:param aws_conn_id: The Airflow connection used for AWS credentials.
|
134
|
+
If this is ``None`` or empty then the default boto3 behaviour is used. If
|
135
|
+
running Airflow in a distributed manner and aws_conn_id is None or
|
136
|
+
empty, then default boto3 configuration would be used (and must be
|
137
|
+
maintained on each worker node).
|
138
|
+
:param region_name: AWS region_name. If not specified then the default boto3 behaviour is used.
|
139
|
+
:param verify: Whether to verify SSL certificates. See:
|
140
|
+
https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html
|
141
|
+
:param botocore_config: Configuration dictionary (key-values) for botocore client. See:
|
142
|
+
https://botocore.amazonaws.com/v1/documentation/api/latest/reference/config.html
|
143
|
+
"""
|
144
|
+
|
145
|
+
aws_hook_class = KinesisAnalyticsV2Hook
|
146
|
+
ui_color = "#44b5e2"
|
147
|
+
|
148
|
+
template_fields: Sequence[str] = aws_template_fields(
|
149
|
+
"application_name",
|
150
|
+
"run_configuration",
|
151
|
+
)
|
152
|
+
template_fields_renderers: dict = {
|
153
|
+
"run_configuration": "json",
|
154
|
+
}
|
155
|
+
|
156
|
+
def __init__(
|
157
|
+
self,
|
158
|
+
application_name: str,
|
159
|
+
run_configuration: dict[str, Any] | None = None,
|
160
|
+
wait_for_completion: bool = True,
|
161
|
+
waiter_delay: int = 60,
|
162
|
+
waiter_max_attempts: int = 20,
|
163
|
+
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
|
164
|
+
**kwargs,
|
165
|
+
):
|
166
|
+
super().__init__(**kwargs)
|
167
|
+
self.application_name = application_name
|
168
|
+
self.run_configuration = run_configuration or {}
|
169
|
+
self.wait_for_completion = wait_for_completion
|
170
|
+
self.waiter_delay = waiter_delay
|
171
|
+
self.waiter_max_attempts = waiter_max_attempts
|
172
|
+
self.deferrable = deferrable
|
173
|
+
|
174
|
+
def execute(self, context: Context) -> dict[str, Any]:
|
175
|
+
msg = "AWS Managed Service for Apache Flink application"
|
176
|
+
|
177
|
+
try:
|
178
|
+
self.log.info("Starting %s %s.", msg, self.application_name)
|
179
|
+
self.hook.conn.start_application(
|
180
|
+
ApplicationName=self.application_name, RunConfiguration=self.run_configuration
|
181
|
+
)
|
182
|
+
except ClientError as error:
|
183
|
+
raise AirflowException(
|
184
|
+
f"Failed to start {msg} {self.application_name}: {error.response['Error']['Message']}"
|
185
|
+
)
|
186
|
+
|
187
|
+
describe_response = self.hook.conn.describe_application(ApplicationName=self.application_name)
|
188
|
+
|
189
|
+
if self.deferrable:
|
190
|
+
self.log.info("Deferring for %s to start: %s.", msg, self.application_name)
|
191
|
+
self.defer(
|
192
|
+
trigger=KinesisAnalyticsV2ApplicationOperationCompleteTrigger(
|
193
|
+
application_name=self.application_name,
|
194
|
+
waiter_name="application_start_complete",
|
195
|
+
aws_conn_id=self.aws_conn_id,
|
196
|
+
waiter_delay=self.waiter_delay,
|
197
|
+
waiter_max_attempts=self.waiter_max_attempts,
|
198
|
+
region_name=self.region_name,
|
199
|
+
verify=self.verify,
|
200
|
+
botocore_config=self.botocore_config,
|
201
|
+
),
|
202
|
+
method_name="execute_complete",
|
203
|
+
)
|
204
|
+
if self.wait_for_completion:
|
205
|
+
self.log.info("Waiting for %s to start: %s.", msg, self.application_name)
|
206
|
+
|
207
|
+
self.hook.get_waiter("application_start_complete").wait(
|
208
|
+
ApplicationName=self.application_name,
|
209
|
+
WaiterConfig={"Delay": self.waiter_delay, "MaxAttempts": self.waiter_max_attempts},
|
210
|
+
)
|
211
|
+
|
212
|
+
self.log.info("%s started successfully %s.", msg, self.application_name)
|
213
|
+
|
214
|
+
return {"ApplicationARN": describe_response["ApplicationDetail"]["ApplicationARN"]}
|
215
|
+
|
216
|
+
def execute_complete(self, context: Context, event: dict[str, Any] | None = None) -> dict[str, Any]:
|
217
|
+
event = validate_execute_complete_event(event)
|
218
|
+
|
219
|
+
if event["status"] != "success":
|
220
|
+
raise AirflowException(
|
221
|
+
"Error while starting AWS Managed Service for Apache Flink application: %s", event
|
222
|
+
)
|
223
|
+
|
224
|
+
response = self.hook.conn.describe_application(
|
225
|
+
ApplicationName=event["application_name"],
|
226
|
+
)
|
227
|
+
|
228
|
+
self.log.info(
|
229
|
+
"AWS Managed Service for Apache Flink application %s started successfully.",
|
230
|
+
event["application_name"],
|
231
|
+
)
|
232
|
+
|
233
|
+
return {"ApplicationARN": response["ApplicationDetail"]["ApplicationARN"]}
|
234
|
+
|
235
|
+
|
236
|
+
class KinesisAnalyticsV2StopApplicationOperator(AwsBaseOperator[KinesisAnalyticsV2Hook]):
|
237
|
+
"""
|
238
|
+
Stop an AWS Managed Service for Apache Flink application.
|
239
|
+
|
240
|
+
.. seealso::
|
241
|
+
For more information on how to use this operator, take a look at the guide:
|
242
|
+
:ref:`howto/operator:KinesisAnalyticsV2StopApplicationOperator`
|
243
|
+
|
244
|
+
:param application_name: The name of your application. (templated)
|
245
|
+
:param force: Set to true to force the application to stop. If you set Force to true, Managed Service for
|
246
|
+
Apache Flink stops the application without taking a snapshot. (templated)
|
247
|
+
|
248
|
+
:param wait_for_completion: Whether to wait for job to stop. (default: True)
|
249
|
+
:param waiter_delay: Time in seconds to wait between status checks. (default: 60)
|
250
|
+
:param waiter_max_attempts: Maximum number of attempts to check for job completion. (default: 20)
|
251
|
+
:param deferrable: If True, the operator will wait asynchronously for the job to stop.
|
252
|
+
This implies waiting for completion. This mode requires aiobotocore module to be installed.
|
253
|
+
(default: False)
|
254
|
+
:param aws_conn_id: The Airflow connection used for AWS credentials.
|
255
|
+
If this is ``None`` or empty then the default boto3 behaviour is used. If
|
256
|
+
running Airflow in a distributed manner and aws_conn_id is None or
|
257
|
+
empty, then default boto3 configuration would be used (and must be
|
258
|
+
maintained on each worker node).
|
259
|
+
:param region_name: AWS region_name. If not specified then the default boto3 behaviour is used.
|
260
|
+
:param verify: Whether to verify SSL certificates. See:
|
261
|
+
https://boto3.amazonaws.com/v1/documentation/api/latest/reference/core/session.html
|
262
|
+
:param botocore_config: Configuration dictionary (key-values) for botocore client. See:
|
263
|
+
https://botocore.amazonaws.com/v1/documentation/api/latest/reference/config.html
|
264
|
+
"""
|
265
|
+
|
266
|
+
aws_hook_class = KinesisAnalyticsV2Hook
|
267
|
+
ui_color = "#44b5e2"
|
268
|
+
|
269
|
+
template_fields: Sequence[str] = aws_template_fields(
|
270
|
+
"application_name",
|
271
|
+
"force",
|
272
|
+
)
|
273
|
+
|
274
|
+
def __init__(
|
275
|
+
self,
|
276
|
+
application_name: str,
|
277
|
+
force: bool = False,
|
278
|
+
wait_for_completion: bool = True,
|
279
|
+
waiter_delay: int = 60,
|
280
|
+
waiter_max_attempts: int = 20,
|
281
|
+
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
|
282
|
+
**kwargs,
|
283
|
+
):
|
284
|
+
super().__init__(**kwargs)
|
285
|
+
self.application_name = application_name
|
286
|
+
self.force = force
|
287
|
+
self.wait_for_completion = wait_for_completion
|
288
|
+
self.waiter_delay = waiter_delay
|
289
|
+
self.waiter_max_attempts = waiter_max_attempts
|
290
|
+
self.deferrable = deferrable
|
291
|
+
|
292
|
+
def execute(self, context: Context) -> dict[str, Any]:
|
293
|
+
msg = "AWS Managed Service for Apache Flink application"
|
294
|
+
|
295
|
+
try:
|
296
|
+
self.log.info("Stopping %s %s.", msg, self.application_name)
|
297
|
+
|
298
|
+
self.hook.conn.stop_application(ApplicationName=self.application_name, Force=self.force)
|
299
|
+
except ClientError as error:
|
300
|
+
raise AirflowException(
|
301
|
+
f"Failed to stop {msg} {self.application_name}: {error.response['Error']['Message']}"
|
302
|
+
)
|
303
|
+
|
304
|
+
describe_response = self.hook.conn.describe_application(ApplicationName=self.application_name)
|
305
|
+
|
306
|
+
if self.deferrable:
|
307
|
+
self.log.info("Deferring for %s to stop: %s.", msg, self.application_name)
|
308
|
+
self.defer(
|
309
|
+
trigger=KinesisAnalyticsV2ApplicationOperationCompleteTrigger(
|
310
|
+
application_name=self.application_name,
|
311
|
+
waiter_name="application_stop_complete",
|
312
|
+
aws_conn_id=self.aws_conn_id,
|
313
|
+
waiter_delay=self.waiter_delay,
|
314
|
+
waiter_max_attempts=self.waiter_max_attempts,
|
315
|
+
region_name=self.region_name,
|
316
|
+
verify=self.verify,
|
317
|
+
botocore_config=self.botocore_config,
|
318
|
+
),
|
319
|
+
method_name="execute_complete",
|
320
|
+
)
|
321
|
+
if self.wait_for_completion:
|
322
|
+
self.log.info("Waiting for %s to stop: %s.", msg, self.application_name)
|
323
|
+
|
324
|
+
self.hook.get_waiter("application_stop_complete").wait(
|
325
|
+
ApplicationName=self.application_name,
|
326
|
+
WaiterConfig={"Delay": self.waiter_delay, "MaxAttempts": self.waiter_max_attempts},
|
327
|
+
)
|
328
|
+
|
329
|
+
self.log.info("%s stopped successfully %s.", msg, self.application_name)
|
330
|
+
|
331
|
+
return {"ApplicationARN": describe_response["ApplicationDetail"]["ApplicationARN"]}
|
332
|
+
|
333
|
+
def execute_complete(self, context: Context, event: dict[str, Any] | None = None) -> dict[str, Any]:
|
334
|
+
event = validate_execute_complete_event(event)
|
335
|
+
|
336
|
+
if event["status"] != "success":
|
337
|
+
raise AirflowException("Error while stopping AWS Managed Service for Apache Flink application")
|
338
|
+
|
339
|
+
response = self.hook.conn.describe_application(
|
340
|
+
ApplicationName=event["application_name"],
|
341
|
+
)
|
342
|
+
|
343
|
+
self.log.info(
|
344
|
+
"AWS Managed Service for Apache Flink application %s stopped successfully.",
|
345
|
+
event["application_name"],
|
346
|
+
)
|
347
|
+
|
348
|
+
return {"ApplicationARN": response["ApplicationDetail"]["ApplicationARN"]}
|
@@ -36,6 +36,7 @@ from airflow.providers.amazon.aws.utils import validate_execute_complete_event
|
|
36
36
|
from airflow.providers.amazon.aws.utils.rds import RdsDbType
|
37
37
|
from airflow.providers.amazon.aws.utils.tags import format_tags
|
38
38
|
from airflow.providers.amazon.aws.utils.waiter_with_logging import wait
|
39
|
+
from airflow.utils.helpers import prune_dict
|
39
40
|
|
40
41
|
if TYPE_CHECKING:
|
41
42
|
from mypy_boto3_rds.type_defs import TagTypeDef
|
@@ -782,7 +783,7 @@ class RdsStartDbOperator(RdsBaseOperator):
|
|
782
783
|
aws_conn_id=self.aws_conn_id,
|
783
784
|
region_name=self.region_name,
|
784
785
|
response=start_db_response,
|
785
|
-
db_type=
|
786
|
+
db_type=self.db_type,
|
786
787
|
),
|
787
788
|
method_name="execute_complete",
|
788
789
|
)
|
@@ -881,12 +882,25 @@ class RdsStopDbOperator(RdsBaseOperator):
|
|
881
882
|
aws_conn_id=self.aws_conn_id,
|
882
883
|
region_name=self.region_name,
|
883
884
|
response=stop_db_response,
|
884
|
-
db_type=
|
885
|
+
db_type=self.db_type,
|
885
886
|
),
|
886
887
|
method_name="execute_complete",
|
887
888
|
)
|
888
889
|
elif self.wait_for_completion:
|
889
|
-
self.
|
890
|
+
waiter = self.hook.get_waiter(f"db_{self.db_type.value}_stopped")
|
891
|
+
waiter_key = (
|
892
|
+
"DBInstanceIdentifier" if self.db_type == RdsDbType.INSTANCE else "DBClusterIdentifier"
|
893
|
+
)
|
894
|
+
kwargs = {waiter_key: self.db_identifier}
|
895
|
+
waiter.wait(
|
896
|
+
WaiterConfig=prune_dict(
|
897
|
+
{
|
898
|
+
"Delay": self.waiter_delay,
|
899
|
+
"MaxAttempts": self.waiter_max_attempts,
|
900
|
+
}
|
901
|
+
),
|
902
|
+
**kwargs,
|
903
|
+
)
|
890
904
|
return json.dumps(stop_db_response, default=str)
|
891
905
|
|
892
906
|
def execute_complete(self, context: Context, event: dict[str, Any] | None = None) -> str:
|
@@ -915,23 +929,6 @@ class RdsStopDbOperator(RdsBaseOperator):
|
|
915
929
|
response = self.hook.conn.stop_db_cluster(DBClusterIdentifier=self.db_identifier)
|
916
930
|
return response
|
917
931
|
|
918
|
-
def _wait_until_db_stopped(self):
|
919
|
-
self.log.info("Waiting for DB %s to reach 'stopped' state", self.db_type.value)
|
920
|
-
if self.db_type == RdsDbType.INSTANCE:
|
921
|
-
self.hook.wait_for_db_instance_state(
|
922
|
-
self.db_identifier,
|
923
|
-
target_state="stopped",
|
924
|
-
check_interval=self.waiter_delay,
|
925
|
-
max_attempts=self.waiter_max_attempts,
|
926
|
-
)
|
927
|
-
else:
|
928
|
-
self.hook.wait_for_db_cluster_state(
|
929
|
-
self.db_identifier,
|
930
|
-
target_state="stopped",
|
931
|
-
check_interval=self.waiter_delay,
|
932
|
-
max_attempts=self.waiter_max_attempts,
|
933
|
-
)
|
934
|
-
|
935
932
|
|
936
933
|
__all__ = [
|
937
934
|
"RdsCreateDbSnapshotOperator",
|
@@ -256,8 +256,6 @@ class RedshiftCreateClusterOperator(BaseOperator):
|
|
256
256
|
params["ClusterVersion"] = self.cluster_version
|
257
257
|
if self.allow_version_upgrade:
|
258
258
|
params["AllowVersionUpgrade"] = self.allow_version_upgrade
|
259
|
-
if self.publicly_accessible:
|
260
|
-
params["PubliclyAccessible"] = self.publicly_accessible
|
261
259
|
if self.encrypted:
|
262
260
|
params["Encrypted"] = self.encrypted
|
263
261
|
if self.hsm_client_certificate_identifier:
|
@@ -287,6 +285,10 @@ class RedshiftCreateClusterOperator(BaseOperator):
|
|
287
285
|
if self.default_iam_role_arn:
|
288
286
|
params["DefaultIamRoleArn"] = self.default_iam_role_arn
|
289
287
|
|
288
|
+
# PubliclyAccessible is True by default on Redshift side, hence, we should always set it regardless
|
289
|
+
# of its value
|
290
|
+
params["PubliclyAccessible"] = self.publicly_accessible
|
291
|
+
|
290
292
|
cluster = redshift_hook.create_cluster(
|
291
293
|
self.cluster_identifier,
|
292
294
|
self.node_type,
|
@@ -324,8 +324,7 @@ class S3CopyObjectOperator(BaseOperator):
|
|
324
324
|
)
|
325
325
|
|
326
326
|
def get_openlineage_facets_on_start(self):
|
327
|
-
from openlineage.
|
328
|
-
|
327
|
+
from airflow.providers.common.compat.openlineage.facet import Dataset
|
329
328
|
from airflow.providers.openlineage.extractors import OperatorLineage
|
330
329
|
|
331
330
|
dest_bucket_name, dest_bucket_key = S3Hook.get_s3_bucket_key(
|
@@ -439,8 +438,7 @@ class S3CreateObjectOperator(BaseOperator):
|
|
439
438
|
s3_hook.load_bytes(self.data, s3_key, s3_bucket, self.replace, self.encrypt, self.acl_policy)
|
440
439
|
|
441
440
|
def get_openlineage_facets_on_start(self):
|
442
|
-
from openlineage.
|
443
|
-
|
441
|
+
from airflow.providers.common.compat.openlineage.facet import Dataset
|
444
442
|
from airflow.providers.openlineage.extractors import OperatorLineage
|
445
443
|
|
446
444
|
bucket, key = S3Hook.get_s3_bucket_key(self.s3_bucket, self.s3_key, "dest_bucket", "dest_key")
|
@@ -546,13 +544,12 @@ class S3DeleteObjectsOperator(BaseOperator):
|
|
546
544
|
|
547
545
|
def get_openlineage_facets_on_complete(self, task_instance):
|
548
546
|
"""Implement _on_complete because object keys are resolved in execute()."""
|
549
|
-
from openlineage.
|
547
|
+
from airflow.providers.common.compat.openlineage.facet import (
|
548
|
+
Dataset,
|
550
549
|
LifecycleStateChange,
|
551
550
|
LifecycleStateChangeDatasetFacet,
|
552
|
-
|
551
|
+
PreviousIdentifier,
|
553
552
|
)
|
554
|
-
from openlineage.client.run import Dataset
|
555
|
-
|
556
553
|
from airflow.providers.openlineage.extractors import OperatorLineage
|
557
554
|
|
558
555
|
if not self._keys:
|
@@ -570,7 +567,7 @@ class S3DeleteObjectsOperator(BaseOperator):
|
|
570
567
|
facets={
|
571
568
|
"lifecycleStateChange": LifecycleStateChangeDatasetFacet(
|
572
569
|
lifecycleStateChange=LifecycleStateChange.DROP.value,
|
573
|
-
previousIdentifier=
|
570
|
+
previousIdentifier=PreviousIdentifier(
|
574
571
|
namespace=bucket_url,
|
575
572
|
name=key,
|
576
573
|
),
|
@@ -725,8 +722,7 @@ class S3FileTransformOperator(BaseOperator):
|
|
725
722
|
self.log.info("Upload successful")
|
726
723
|
|
727
724
|
def get_openlineage_facets_on_start(self):
|
728
|
-
from openlineage.
|
729
|
-
|
725
|
+
from airflow.providers.common.compat.openlineage.facet import Dataset
|
730
726
|
from airflow.providers.openlineage.extractors import OperatorLineage
|
731
727
|
|
732
728
|
dest_bucket_name, dest_bucket_key = S3Hook.get_s3_bucket_key(
|
@@ -46,8 +46,7 @@ from airflow.utils.helpers import prune_dict
|
|
46
46
|
from airflow.utils.json import AirflowJsonEncoder
|
47
47
|
|
48
48
|
if TYPE_CHECKING:
|
49
|
-
from openlineage.
|
50
|
-
|
49
|
+
from airflow.providers.common.compat.openlineage.facet import Dataset
|
51
50
|
from airflow.providers.openlineage.extractors.base import OperatorLineage
|
52
51
|
from airflow.utils.context import Context
|
53
52
|
|
@@ -208,7 +207,7 @@ class SageMakerBaseOperator(BaseOperator):
|
|
208
207
|
|
209
208
|
@staticmethod
|
210
209
|
def path_to_s3_dataset(path) -> Dataset:
|
211
|
-
from openlineage.
|
210
|
+
from airflow.providers.common.compat.openlineage.facet import Dataset
|
212
211
|
|
213
212
|
path = path.replace("s3://", "")
|
214
213
|
split_path = path.split("/")
|
@@ -48,6 +48,8 @@ class StepFunctionStartExecutionOperator(AwsBaseOperator[StepFunctionHook]):
|
|
48
48
|
|
49
49
|
:param state_machine_arn: ARN of the Step Function State Machine
|
50
50
|
:param name: The name of the execution.
|
51
|
+
:param is_redrive_execution: Restarts unsuccessful executions of Standard workflows that did not
|
52
|
+
complete successfully in the last 14 days.
|
51
53
|
:param state_machine_input: JSON data input to pass to the State Machine
|
52
54
|
:param aws_conn_id: The Airflow connection used for AWS credentials.
|
53
55
|
If this is None or empty then the default boto3 behaviour is used. If
|
@@ -73,7 +75,9 @@ class StepFunctionStartExecutionOperator(AwsBaseOperator[StepFunctionHook]):
|
|
73
75
|
"""
|
74
76
|
|
75
77
|
aws_hook_class = StepFunctionHook
|
76
|
-
template_fields: Sequence[str] = aws_template_fields(
|
78
|
+
template_fields: Sequence[str] = aws_template_fields(
|
79
|
+
"state_machine_arn", "name", "input", "is_redrive_execution"
|
80
|
+
)
|
77
81
|
ui_color = "#f9c915"
|
78
82
|
operator_extra_links = (StateMachineDetailsLink(), StateMachineExecutionsDetailsLink())
|
79
83
|
|
@@ -82,6 +86,7 @@ class StepFunctionStartExecutionOperator(AwsBaseOperator[StepFunctionHook]):
|
|
82
86
|
*,
|
83
87
|
state_machine_arn: str,
|
84
88
|
name: str | None = None,
|
89
|
+
is_redrive_execution: bool = False,
|
85
90
|
state_machine_input: dict | str | None = None,
|
86
91
|
waiter_max_attempts: int = 30,
|
87
92
|
waiter_delay: int = 60,
|
@@ -91,6 +96,7 @@ class StepFunctionStartExecutionOperator(AwsBaseOperator[StepFunctionHook]):
|
|
91
96
|
super().__init__(**kwargs)
|
92
97
|
self.state_machine_arn = state_machine_arn
|
93
98
|
self.name = name
|
99
|
+
self.is_redrive_execution = is_redrive_execution
|
94
100
|
self.input = state_machine_input
|
95
101
|
self.waiter_delay = waiter_delay
|
96
102
|
self.waiter_max_attempts = waiter_max_attempts
|
@@ -105,7 +111,11 @@ class StepFunctionStartExecutionOperator(AwsBaseOperator[StepFunctionHook]):
|
|
105
111
|
state_machine_arn=self.state_machine_arn,
|
106
112
|
)
|
107
113
|
|
108
|
-
if not (
|
114
|
+
if not (
|
115
|
+
execution_arn := self.hook.start_execution(
|
116
|
+
self.state_machine_arn, self.name, self.input, self.is_redrive_execution
|
117
|
+
)
|
118
|
+
):
|
109
119
|
raise AirflowException(f"Failed to start State Machine execution for: {self.state_machine_arn}")
|
110
120
|
|
111
121
|
StateMachineExecutionsDetailsLink.persist(
|