From 525c9173a7db0e7cbe337a2252204b2d076124cf Mon Sep 17 00:00:00 2001 From: Chris Guidry Date: Wed, 4 Oct 2023 15:48:55 -0400 Subject: [PATCH 01/11] Conditional imports to support operating with `pydantic>2` installed (#317) --- prefect_aws/client_parameters.py | 7 ++++++- prefect_aws/credentials.py | 7 ++++++- prefect_aws/ecs.py | 8 +++++++- prefect_aws/s3.py | 7 ++++++- prefect_aws/secrets_manager.py | 7 ++++++- prefect_aws/workers/ecs_worker.py | 8 +++++++- requirements-dev.txt | 4 +++- tests/test_ecs.py | 7 ++++++- tests/workers/test_ecs_worker.py | 8 +++++++- 9 files changed, 54 insertions(+), 9 deletions(-) diff --git a/prefect_aws/client_parameters.py b/prefect_aws/client_parameters.py index 3f02b4b2..bf030590 100644 --- a/prefect_aws/client_parameters.py +++ b/prefect_aws/client_parameters.py @@ -5,7 +5,12 @@ from botocore import UNSIGNED from botocore.client import Config -from pydantic import BaseModel, Field, FilePath, root_validator, validator +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import BaseModel, Field, FilePath, root_validator, validator +else: + from pydantic import BaseModel, Field, FilePath, root_validator, validator class AwsClientParameters(BaseModel): diff --git a/prefect_aws/credentials.py b/prefect_aws/credentials.py index f390faef..7810f7f9 100644 --- a/prefect_aws/credentials.py +++ b/prefect_aws/credentials.py @@ -7,7 +7,12 @@ from mypy_boto3_s3 import S3Client from mypy_boto3_secretsmanager import SecretsManagerClient from prefect.blocks.abstract import CredentialsBlock -from pydantic import Field, SecretStr +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import Field, SecretStr +else: + from pydantic import Field, SecretStr from prefect_aws.client_parameters import AwsClientParameters diff --git a/prefect_aws/ecs.py b/prefect_aws/ecs.py index a9534332..7dbfa68a 100644 --- a/prefect_aws/ecs.py +++ b/prefect_aws/ecs.py @@ -121,7 +121,13 @@ from prefect.utilities.asyncutils import run_sync_in_worker_thread, sync_compatible from prefect.utilities.dockerutils import get_prefect_image_name from prefect.utilities.pydantic import JsonPatch -from pydantic import Field, root_validator, validator +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import Field, root_validator, validator +else: + from pydantic import Field, root_validator, validator + from slugify import slugify from typing_extensions import Literal, Self diff --git a/prefect_aws/s3.py b/prefect_aws/s3.py index 17fe8870..fe646ddc 100644 --- a/prefect_aws/s3.py +++ b/prefect_aws/s3.py @@ -14,7 +14,12 @@ from prefect.filesystems import WritableDeploymentStorage, WritableFileSystem from prefect.utilities.asyncutils import run_sync_in_worker_thread, sync_compatible from prefect.utilities.filesystem import filter_files -from pydantic import Field +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import Field +else: + from pydantic import Field from prefect_aws import AwsCredentials, MinIOCredentials from prefect_aws.client_parameters import AwsClientParameters diff --git a/prefect_aws/secrets_manager.py b/prefect_aws/secrets_manager.py index 01bc9ba1..5eea5ba8 100644 --- a/prefect_aws/secrets_manager.py +++ b/prefect_aws/secrets_manager.py @@ -5,7 +5,12 @@ from prefect import get_run_logger, task from prefect.blocks.abstract import SecretBlock from prefect.utilities.asyncutils import run_sync_in_worker_thread, sync_compatible -from pydantic import Field +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import Field +else: + from pydantic import Field from prefect_aws import AwsCredentials diff --git a/prefect_aws/workers/ecs_worker.py b/prefect_aws/workers/ecs_worker.py index f4d97bef..1b1c8073 100644 --- a/prefect_aws/workers/ecs_worker.py +++ b/prefect_aws/workers/ecs_worker.py @@ -68,7 +68,13 @@ BaseWorker, BaseWorkerResult, ) -from pydantic import Field, root_validator +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import Field, root_validator +else: + from pydantic import Field, root_validator + from slugify import slugify from tenacity import retry, stop_after_attempt, wait_fixed, wait_random from typing_extensions import Literal diff --git a/requirements-dev.txt b/requirements-dev.txt index 9003b759..e22e9290 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -9,7 +9,9 @@ mkdocs-gen-files mkdocs-material mkdocstrings-python-legacy mock; python_version < '3.8' -moto >= 3.1.16 +# moto 4.2.5 broke something fairly deep in our test suite +# https://github.com/PrefectHQ/prefect-aws/issues/318 +moto >= 3.1.16, < 4.2.5 mypy pillow pre-commit diff --git a/tests/test_ecs.py b/tests/test_ecs.py index 9282a859..a27ff925 100644 --- a/tests/test_ecs.py +++ b/tests/test_ecs.py @@ -16,7 +16,12 @@ from prefect.server.schemas.core import Deployment, Flow, FlowRun from prefect.utilities.asyncutils import run_sync_in_worker_thread from prefect.utilities.dockerutils import get_prefect_image_name -from pydantic import ValidationError +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import ValidationError +else: + from pydantic import ValidationError from prefect_aws.ecs import ( ECS_DEFAULT_CPU, diff --git a/tests/workers/test_ecs_worker.py b/tests/workers/test_ecs_worker.py index ef57372b..1be6eb70 100644 --- a/tests/workers/test_ecs_worker.py +++ b/tests/workers/test_ecs_worker.py @@ -12,7 +12,13 @@ from moto.ec2.utils import generate_instance_identity_document from prefect.server.schemas.core import FlowRun from prefect.utilities.asyncutils import run_sync_in_worker_thread -from pydantic import ValidationError +from pydantic import VERSION as PYDANTIC_VERSION + +if PYDANTIC_VERSION.startswith("2."): + from pydantic.v1 import ValidationError +else: + from pydantic import ValidationError + from tenacity import RetryError from prefect_aws.workers.ecs_worker import ( From 8650dbfba1d11a9fa823a81e577202b2d178ac79 Mon Sep 17 00:00:00 2001 From: rc <4758618+chotalia@users.noreply.github.com> Date: Wed, 11 Oct 2023 07:01:47 +1100 Subject: [PATCH 02/11] fix: 320 Subnet validation in Network Configuration fails when multiple subnets exist in the VPC (#321) --- CHANGELOG.md | 4 + prefect_aws/workers/ecs_worker.py | 6 +- tests/workers/test_ecs_worker.py | 145 +++++++++++++++++++++++++++++- 3 files changed, 151 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7967e69f..f3cc4f89 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,10 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Changed `push_to_s3` deployment step function to write paths `as_posix()` to allow support for deploying from windows [#314](https://github.com/PrefectHQ/prefect-aws/pull/314) +### Fixed + +- Resolved an issue where defining a custom network configuration with a subnet would erroneously report it as missing from the VPC when more than one subnet exists in the VPC. [#321](https://github.com/PrefectHQ/prefect-aws/pull/321) + ### Deprecated ### Removed diff --git a/prefect_aws/workers/ecs_worker.py b/prefect_aws/workers/ecs_worker.py index 1b1c8073..a3822a3f 100644 --- a/prefect_aws/workers/ecs_worker.py +++ b/prefect_aws/workers/ecs_worker.py @@ -1349,10 +1349,10 @@ def _custom_network_configuration( + "Network configuration cannot be inferred." ) + subnet_ids = [subnet["SubnetId"] for subnet in subnets] + config_subnets = network_configuration.get("subnets", []) - if not all( - [conf_sn in sn.values() for conf_sn in config_subnets for sn in subnets] - ): + if not all(conf_sn in subnet_ids for conf_sn in config_subnets): raise ValueError( f"Subnets {config_subnets} not found within {vpc_message}." + "Please check that VPC is associated with supplied subnets." diff --git a/tests/workers/test_ecs_worker.py b/tests/workers/test_ecs_worker.py index 1be6eb70..2c177aae 100644 --- a/tests/workers/test_ecs_worker.py +++ b/tests/workers/test_ecs_worker.py @@ -892,7 +892,7 @@ async def test_network_config_from_vpc_id( @pytest.mark.usefixtures("ecs_mocks") -async def test_network_config_from_custom_settings( +async def test_network_config_1_subnet_in_custom_settings_1_in_vpc( aws_credentials: AwsCredentials, flow_run: FlowRun ): session = aws_credentials.get_boto3_session() @@ -937,6 +937,107 @@ async def test_network_config_from_custom_settings( } +@pytest.mark.usefixtures("ecs_mocks") +async def test_network_config_1_sn_in_custom_settings_many_in_vpc( + aws_credentials: AwsCredentials, flow_run: FlowRun +): + session = aws_credentials.get_boto3_session() + ec2_resource = session.resource("ec2") + vpc = ec2_resource.create_vpc(CidrBlock="10.0.0.0/16") + subnet = ec2_resource.create_subnet(CidrBlock="10.0.2.0/24", VpcId=vpc.id) + ec2_resource.create_subnet(CidrBlock="10.0.3.0/24", VpcId=vpc.id) + ec2_resource.create_subnet(CidrBlock="10.0.4.0/24", VpcId=vpc.id) + + security_group = ec2_resource.create_security_group( + GroupName="ECSWorkerTestSG", Description="ECS Worker test SG", VpcId=vpc.id + ) + + configuration = await construct_configuration( + aws_credentials=aws_credentials, + vpc_id=vpc.id, + override_network_configuration=True, + network_configuration={ + "subnets": [subnet.id], + "assignPublicIp": "DISABLED", + "securityGroups": [security_group.id], + }, + ) + + session = aws_credentials.get_boto3_session() + + async with ECSWorker(work_pool_name="test") as worker: + # Capture the task run call because moto does not track 'networkConfiguration' + original_run_task = worker._create_task_run + mock_run_task = MagicMock(side_effect=original_run_task) + worker._create_task_run = mock_run_task + + result = await run_then_stop_task(worker, configuration, flow_run) + + assert result.status_code == 0 + network_configuration = mock_run_task.call_args[0][1].get("networkConfiguration") + + # Subnet ids are copied from the vpc + assert network_configuration == { + "awsvpcConfiguration": { + "subnets": [subnet.id], + "assignPublicIp": "DISABLED", + "securityGroups": [security_group.id], + } + } + + +@pytest.mark.usefixtures("ecs_mocks") +async def test_network_config_many_subnet_in_custom_settings_many_in_vpc( + aws_credentials: AwsCredentials, flow_run: FlowRun +): + session = aws_credentials.get_boto3_session() + ec2_resource = session.resource("ec2") + vpc = ec2_resource.create_vpc(CidrBlock="10.0.0.0/16") + subnets = [ + ec2_resource.create_subnet(CidrBlock="10.0.2.0/24", VpcId=vpc.id), + ec2_resource.create_subnet(CidrBlock="10.0.33.0/24", VpcId=vpc.id), + ec2_resource.create_subnet(CidrBlock="10.0.44.0/24", VpcId=vpc.id), + ] + subnet_ids = [subnet.id for subnet in subnets] + + security_group = ec2_resource.create_security_group( + GroupName="ECSWorkerTestSG", Description="ECS Worker test SG", VpcId=vpc.id + ) + + configuration = await construct_configuration( + aws_credentials=aws_credentials, + vpc_id=vpc.id, + override_network_configuration=True, + network_configuration={ + "subnets": subnet_ids, + "assignPublicIp": "DISABLED", + "securityGroups": [security_group.id], + }, + ) + + session = aws_credentials.get_boto3_session() + + async with ECSWorker(work_pool_name="test") as worker: + # Capture the task run call because moto does not track 'networkConfiguration' + original_run_task = worker._create_task_run + mock_run_task = MagicMock(side_effect=original_run_task) + worker._create_task_run = mock_run_task + + result = await run_then_stop_task(worker, configuration, flow_run) + + assert result.status_code == 0 + network_configuration = mock_run_task.call_args[0][1].get("networkConfiguration") + + # Subnet ids are copied from the vpc + assert network_configuration == { + "awsvpcConfiguration": { + "subnets": subnet_ids, + "assignPublicIp": "DISABLED", + "securityGroups": [security_group.id], + } + } + + @pytest.mark.usefixtures("ecs_mocks") async def test_network_config_from_custom_settings_invalid_subnet( aws_credentials: AwsCredentials, flow_run: FlowRun @@ -978,6 +1079,48 @@ async def test_network_config_from_custom_settings_invalid_subnet( await run_then_stop_task(worker, configuration, flow_run) +@pytest.mark.usefixtures("ecs_mocks") +async def test_network_config_from_custom_settings_invalid_subnet_multiple_vpc_subnets( + aws_credentials: AwsCredentials, flow_run: FlowRun +): + session = aws_credentials.get_boto3_session() + ec2_resource = session.resource("ec2") + vpc = ec2_resource.create_vpc(CidrBlock="10.0.0.0/16") + security_group = ec2_resource.create_security_group( + GroupName="ECSWorkerTestSG", Description="ECS Worker test SG", VpcId=vpc.id + ) + subnet = ec2_resource.create_subnet(CidrBlock="10.0.2.0/24", VpcId=vpc.id) + invalid_subnet_id = "subnet-3bf19de7" + + configuration = await construct_configuration( + aws_credentials=aws_credentials, + vpc_id=vpc.id, + override_network_configuration=True, + network_configuration={ + "subnets": [invalid_subnet_id, subnet.id], + "assignPublicIp": "DISABLED", + "securityGroups": [security_group.id], + }, + ) + + session = aws_credentials.get_boto3_session() + + with pytest.raises( + ValueError, + match=( + rf"Subnets \['{invalid_subnet_id}', '{subnet.id}'\] not found within VPC" + f" with ID {vpc.id}.Please check that VPC is associated with supplied" + " subnets." + ), + ): + async with ECSWorker(work_pool_name="test") as worker: + original_run_task = worker._create_task_run + mock_run_task = MagicMock(side_effect=original_run_task) + worker._create_task_run = mock_run_task + + await run_then_stop_task(worker, configuration, flow_run) + + @pytest.mark.usefixtures("ecs_mocks") async def test_network_config_configure_network_requires_vpc_id( aws_credentials: AwsCredentials, flow_run: FlowRun From 9ba54243a7b149c838edfdb8c7f80653b98d7df2 Mon Sep 17 00:00:00 2001 From: Dominic Tarro <57306102+dominictarro@users.noreply.github.com> Date: Tue, 10 Oct 2023 16:09:46 -0400 Subject: [PATCH 03/11] AWS S3 copy and move tasks and `S3Bucket` methods (#316) --- CHANGELOG.md | 2 + prefect_aws/s3.py | 325 +++++++++++++++++++++++++++++++++++++++++++++- tests/test_s3.py | 238 ++++++++++++++++++++++++++++++++- 3 files changed, 561 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f3cc4f89..c73c15ff 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## Unreleased +- AWS S3 copy and move tasks and `S3Bucket` methods - [#316](https://github.com/PrefectHQ/prefect-aws/pull/316) + ### Added - Added retries to ECS task run creation for ECS worker - [#303](https://github.com/PrefectHQ/prefect-aws/pull/303) diff --git a/prefect_aws/s3.py b/prefect_aws/s3.py index fe646ddc..df84873c 100644 --- a/prefect_aws/s3.py +++ b/prefect_aws/s3.py @@ -152,6 +152,160 @@ async def example_s3_upload_flow(): return key +@task +async def s3_copy( + source_path: str, + target_path: str, + source_bucket_name: str, + aws_credentials: AwsCredentials, + target_bucket_name: Optional[str] = None, + **copy_kwargs, +) -> str: + """Uses S3's internal + [CopyObject](https://docs.aws.amazon.com/AmazonS3/latest/API/API_CopyObject.html) + to copy objects within or between buckets. To copy objects between buckets, the + credentials must have permission to read the source object and write to the target + object. If the credentials do not have those permissions, try using + `S3Bucket.stream_from`. + + Args: + source_path: The path to the object to copy. Can be a string or `Path`. + target_path: The path to copy the object to. Can be a string or `Path`. + source_bucket_name: The bucket to copy the object from. + aws_credentials: Credentials to use for authentication with AWS. + target_bucket_name: The bucket to copy the object to. If not provided, defaults + to `source_bucket`. + **copy_kwargs: Additional keyword arguments to pass to `S3Client.copy_object`. + + Returns: + The path that the object was copied to. Excludes the bucket name. + + Examples: + + Copy notes.txt from s3://my-bucket/my_folder/notes.txt to + s3://my-bucket/my_folder/notes_copy.txt. + + ```python + from prefect import flow + from prefect_aws import AwsCredentials + from prefect_aws.s3 import s3_copy + + aws_credentials = AwsCredentials.load("my-creds") + + @flow + async def example_copy_flow(): + await s3_copy( + source_path="my_folder/notes.txt", + target_path="my_folder/notes_copy.txt", + source_bucket_name="my-bucket", + aws_credentials=aws_credentials, + ) + + example_copy_flow() + ``` + + Copy notes.txt from s3://my-bucket/my_folder/notes.txt to + s3://other-bucket/notes_copy.txt. + + ```python + from prefect import flow + from prefect_aws import AwsCredentials + from prefect_aws.s3 import s3_copy + + aws_credentials = AwsCredentials.load("shared-creds") + + @flow + async def example_copy_flow(): + await s3_copy( + source_path="my_folder/notes.txt", + target_path="notes_copy.txt", + source_bucket_name="my-bucket", + aws_credentials=aws_credentials, + target_bucket_name="other-bucket", + ) + + example_copy_flow() + ``` + + """ + logger = get_run_logger() + + s3_client = aws_credentials.get_s3_client() + + target_bucket_name = target_bucket_name or source_bucket_name + + logger.info( + "Copying object from bucket %s with key %s to bucket %s with key %s", + source_bucket_name, + source_path, + target_bucket_name, + target_path, + ) + + s3_client.copy_object( + CopySource={"Bucket": source_bucket_name, "Key": source_path}, + Bucket=target_bucket_name, + Key=target_path, + **copy_kwargs, + ) + + return target_path + + +@task +async def s3_move( + source_path: str, + target_path: str, + source_bucket_name: str, + aws_credentials: AwsCredentials, + target_bucket_name: Optional[str] = None, +) -> str: + """ + Move an object from one S3 location to another. To move objects between buckets, + the credentials must have permission to read and delete the source object and write + to the target object. If the credentials do not have those permissions, this method + will raise an error. If the credentials have permission to read the source object + but not delete it, the object will be copied but not deleted. + + Args: + source_path: The path of the object to move + target_path: The path to move the object to + source_bucket_name: The name of the bucket containing the source object + aws_credentials: Credentials to use for authentication with AWS. + target_bucket_name: The bucket to copy the object to. If not provided, defaults + to `source_bucket`. + + Returns: + The path that the object was moved to. Excludes the bucket name. + """ + logger = get_run_logger() + + s3_client = aws_credentials.get_s3_client() + + # If target bucket is not provided, assume it's the same as the source bucket + target_bucket_name = target_bucket_name or source_bucket_name + + logger.info( + "Moving object from s3://%s/%s s3://%s/%s", + source_bucket_name, + source_path, + target_bucket_name, + target_path, + ) + + # Copy the object to the new location + s3_client.copy_object( + Bucket=target_bucket_name, + CopySource={"Bucket": source_bucket_name, "Key": source_path}, + Key=target_path, + ) + + # Delete the original object + s3_client.delete_object(Bucket=source_bucket_name, Key=source_path) + + return target_path + + def _list_objects_sync(page_iterator: PageIterator): """ Synchronous method to collect S3 objects into a list @@ -800,7 +954,9 @@ async def stream_from( to_path: Optional[str] = None, **upload_kwargs: Dict[str, Any], ) -> str: - """Streams an object from another bucket to this bucket. + """Streams an object from another bucket to this bucket. Requires the + object to be downloaded and uploaded in chunks. If `self`'s credentials + allow for writes to the other bucket, try using `S3Bucket.copy_object`. Args: bucket: The bucket to stream from. @@ -1028,3 +1184,170 @@ async def upload_from_folder( ) return to_folder + + @sync_compatible + async def copy_object( + self, + from_path: Union[str, Path], + to_path: Union[str, Path], + to_bucket: Optional[Union["S3Bucket", str]] = None, + **copy_kwargs, + ) -> str: + """Uses S3's internal + [CopyObject](https://docs.aws.amazon.com/AmazonS3/latest/API/API_CopyObject.html) + to copy objects within or between buckets. To copy objects between buckets, + `self`'s credentials must have permission to read the source object and write + to the target object. If the credentials do not have those permissions, try + using `S3Bucket.stream_from`. + + Args: + from_path: The path of the object to copy. + to_path: The path to copy the object to. + to_bucket: The bucket to copy to. Defaults to the current bucket. + **copy_kwargs: Additional keyword arguments to pass to + `S3Client.copy_object`. + + Returns: + The path that the object was copied to. Excludes the bucket name. + + Examples: + + Copy notes.txt from my_folder/notes.txt to my_folder/notes_copy.txt. + + ```python + from prefect_aws.s3 import S3Bucket + + s3_bucket = S3Bucket.load("my-bucket") + s3_bucket.copy_object("my_folder/notes.txt", "my_folder/notes_copy.txt") + ``` + + Copy notes.txt from my_folder/notes.txt to my_folder/notes_copy.txt in + another bucket. + + ```python + from prefect_aws.s3 import S3Bucket + + s3_bucket = S3Bucket.load("my-bucket") + s3_bucket.copy_object( + "my_folder/notes.txt", + "my_folder/notes_copy.txt", + to_bucket="other-bucket" + ) + ``` + """ + s3_client = self.credentials.get_s3_client() + + source_path = self._resolve_path(Path(from_path).as_posix()) + target_path = self._resolve_path(Path(to_path).as_posix()) + + source_bucket_name = self.bucket_name + target_bucket_name = self.bucket_name + if isinstance(to_bucket, S3Bucket): + target_bucket_name = to_bucket.bucket_name + target_path = to_bucket._resolve_path(target_path) + elif isinstance(to_bucket, str): + target_bucket_name = to_bucket + elif to_bucket is not None: + raise TypeError( + "to_bucket must be a string or S3Bucket, not" + f" {type(target_bucket_name)}" + ) + + self.logger.info( + "Copying object from bucket %s with key %s to bucket %s with key %s", + source_bucket_name, + source_path, + target_bucket_name, + target_path, + ) + + s3_client.copy_object( + CopySource={"Bucket": source_bucket_name, "Key": source_path}, + Bucket=target_bucket_name, + Key=target_path, + **copy_kwargs, + ) + + return target_path + + @sync_compatible + async def move_object( + self, + from_path: Union[str, Path], + to_path: Union[str, Path], + to_bucket: Optional[Union["S3Bucket", str]] = None, + ) -> str: + """Uses S3's internal CopyObject and DeleteObject to move objects within or + between buckets. To move objects between buckets, `self`'s credentials must + have permission to read and delete the source object and write to the target + object. If the credentials do not have those permissions, this method will + raise an error. If the credentials have permission to read the source object + but not delete it, the object will be copied but not deleted. + + Args: + from_path: The path of the object to move. + to_path: The path to move the object to. + to_bucket: The bucket to move to. Defaults to the current bucket. + + Returns: + The path that the object was moved to. Excludes the bucket name. + + Examples: + + Move notes.txt from my_folder/notes.txt to my_folder/notes_copy.txt. + + ```python + from prefect_aws.s3 import S3Bucket + + s3_bucket = S3Bucket.load("my-bucket") + s3_bucket.move_object("my_folder/notes.txt", "my_folder/notes_copy.txt") + ``` + + Move notes.txt from my_folder/notes.txt to my_folder/notes_copy.txt in + another bucket. + + ```python + from prefect_aws.s3 import S3Bucket + + s3_bucket = S3Bucket.load("my-bucket") + s3_bucket.move_object( + "my_folder/notes.txt", + "my_folder/notes_copy.txt", + to_bucket="other-bucket" + ) + ``` + """ + s3_client = self.credentials.get_s3_client() + + source_path = self._resolve_path(Path(from_path).as_posix()) + target_path = self._resolve_path(Path(to_path).as_posix()) + + source_bucket_name = self.bucket_name + target_bucket_name = self.bucket_name + if isinstance(to_bucket, S3Bucket): + target_bucket_name = to_bucket.bucket_name + target_path = to_bucket._resolve_path(target_path) + elif isinstance(to_bucket, str): + target_bucket_name = to_bucket + elif to_bucket is not None: + raise TypeError( + "to_bucket must be a string or S3Bucket, not" + f" {type(target_bucket_name)}" + ) + + self.logger.info( + "Moving object from s3://%s/%s to s3://%s/%s", + source_bucket_name, + source_path, + target_bucket_name, + target_path, + ) + + # If invalid, should error and prevent next operation + s3_client.copy( + CopySource={"Bucket": source_bucket_name, "Key": source_path}, + Bucket=target_bucket_name, + Key=target_path, + ) + s3_client.delete_object(Bucket=source_bucket_name, Key=source_path) + return target_path diff --git a/tests/test_s3.py b/tests/test_s3.py index c0159e25..89a39f7d 100644 --- a/tests/test_s3.py +++ b/tests/test_s3.py @@ -12,7 +12,14 @@ from prefect_aws import AwsCredentials, MinIOCredentials from prefect_aws.client_parameters import AwsClientParameters -from prefect_aws.s3 import S3Bucket, s3_download, s3_list_objects, s3_upload +from prefect_aws.s3 import ( + S3Bucket, + s3_copy, + s3_download, + s3_list_objects, + s3_move, + s3_upload, +) aws_clients = [ (lazy_fixture("aws_client_parameters_custom_endpoint")), @@ -47,6 +54,18 @@ def bucket(s3_mock, request): return bucket +@pytest.fixture +def bucket_2(s3_mock, request): + s3 = boto3.resource("s3") + bucket = s3.Bucket("bucket_2") + marker = request.node.get_closest_marker("is_public", None) + if marker and marker.args[0]: + bucket.create(ACL="public-read") + else: + bucket.create() + return bucket + + @pytest.fixture def object(bucket, tmp_path): file = tmp_path / "object.txt" @@ -205,6 +224,151 @@ async def test_flow(): assert output == b"NEW OBJECT" +@pytest.mark.parametrize("client_parameters", aws_clients, indirect=True) +async def test_s3_copy(object, bucket, bucket_2, aws_credentials): + def read(bucket, key): + stream = io.BytesIO() + bucket.download_fileobj(key, stream) + stream.seek(0) + return stream.read() + + @flow + async def test_flow(): + # Test cross-bucket copy + await s3_copy( + source_path="object", + target_path="subfolder/new_object", + source_bucket_name="bucket", + aws_credentials=aws_credentials, + target_bucket_name="bucket_2", + ) + + # Test within-bucket copy + await s3_copy( + source_path="object", + target_path="subfolder/new_object", + source_bucket_name="bucket", + aws_credentials=aws_credentials, + ) + + await test_flow() + assert read(bucket_2, "subfolder/new_object") == b"TEST" + assert read(bucket, "subfolder/new_object") == b"TEST" + + +@pytest.mark.parametrize("client_parameters", aws_clients, indirect=True) +async def test_s3_move(object, bucket, bucket_2, aws_credentials): + def read(bucket, key): + stream = io.BytesIO() + bucket.download_fileobj(key, stream) + stream.seek(0) + return stream.read() + + @flow + async def test_flow(): + # Test within-bucket move + await s3_move( + source_path="object", + target_path="subfolder/object_copy", + source_bucket_name="bucket", + aws_credentials=aws_credentials, + ) + + # Test cross-bucket move + await s3_move( + source_path="subfolder/object_copy", + target_path="object_copy_2", + source_bucket_name="bucket", + target_bucket_name="bucket_2", + aws_credentials=aws_credentials, + ) + + await test_flow() + + assert read(bucket_2, "object_copy_2") == b"TEST" + + with pytest.raises(ClientError): + read(bucket, "object") + + with pytest.raises(ClientError): + read(bucket, "subfolder/object_copy") + + +@pytest.mark.parametrize("client_parameters", aws_clients, indirect=True) +async def test_move_object_to_nonexistent_bucket_fails( + object, + bucket, + aws_credentials, +): + def read(bucket, key): + stream = io.BytesIO() + bucket.download_fileobj(key, stream) + stream.seek(0) + return stream.read() + + @flow + async def test_flow(): + # Test cross-bucket move + await s3_move( + source_path="object", + target_path="subfolder/new_object", + source_bucket_name="bucket", + aws_credentials=aws_credentials, + target_bucket_name="nonexistent-bucket", + ) + + with pytest.raises(ClientError): + await test_flow() + + assert read(bucket, "object") == b"TEST" + + +@pytest.mark.parametrize("client_parameters", aws_clients, indirect=True) +async def test_move_object_fail_cases( + object, + bucket, + aws_credentials, +): + def read(bucket, key): + stream = io.BytesIO() + bucket.download_fileobj(key, stream) + stream.seek(0) + return stream.read() + + @flow + async def test_flow( + source_path, target_path, source_bucket_name, target_bucket_name + ): + # Test cross-bucket move + await s3_move( + source_path=source_path, + target_path=target_path, + source_bucket_name=source_bucket_name, + aws_credentials=aws_credentials, + target_bucket_name=target_bucket_name, + ) + + # Move to non-existent bucket + with pytest.raises(ClientError): + await test_flow( + source_path="object", + target_path="subfolder/new_object", + source_bucket_name="bucket", + target_bucket_name="nonexistent-bucket", + ) + assert read(bucket, "object") == b"TEST" + + # Move onto self + with pytest.raises(ClientError): + await test_flow( + source_path="object", + target_path="object", + source_bucket_name="bucket", + target_bucket_name="bucket", + ) + assert read(bucket, "object") == b"TEST" + + @pytest.mark.parametrize("client_parameters", aws_clients, indirect=True) async def test_s3_list_objects( object, client_parameters, object_in_folder, aws_credentials @@ -623,9 +787,9 @@ def s3_bucket_empty(self, credentials, bucket): return _s3_bucket @pytest.fixture - def s3_bucket_2_empty(self, credentials, bucket): + def s3_bucket_2_empty(self, credentials, bucket_2): _s3_bucket = S3Bucket( - bucket_name="bucket", + bucket_name="bucket_2", credentials=credentials, bucket_folder="subfolder", ) @@ -811,3 +975,71 @@ def test_upload_from_folder( break else: raise AssertionError("Files did upload") + + @pytest.mark.parametrize("client_parameters", aws_clients[-1:], indirect=True) + def test_copy_object( + self, + s3_bucket_with_object: S3Bucket, + s3_bucket_2_empty: S3Bucket, + ): + s3_bucket_with_object.copy_object("object", "object_copy_1") + assert s3_bucket_with_object.read_path("object_copy_1") == b"TEST" + + s3_bucket_with_object.copy_object("object", "folder/object_copy_2") + assert s3_bucket_with_object.read_path("folder/object_copy_2") == b"TEST" + + # S3Bucket for second bucket has a basepath + s3_bucket_with_object.copy_object( + "object", + s3_bucket_2_empty._resolve_path("object_copy_3"), + to_bucket="bucket_2", + ) + assert s3_bucket_2_empty.read_path("object_copy_3") == b"TEST" + + s3_bucket_with_object.copy_object("object", "object_copy_4", s3_bucket_2_empty) + assert s3_bucket_2_empty.read_path("object_copy_4") == b"TEST" + + @pytest.mark.parametrize("client_parameters", aws_clients[-1:], indirect=True) + def test_move_object_within_bucket( + self, + s3_bucket_with_object: S3Bucket, + ): + s3_bucket_with_object.move_object("object", "object_copy_1") + assert s3_bucket_with_object.read_path("object_copy_1") == b"TEST" + + with pytest.raises(ClientError): + assert s3_bucket_with_object.read_path("object") == b"TEST" + + @pytest.mark.parametrize("client_parameters", aws_clients[-1:], indirect=True) + def test_move_object_to_nonexistent_bucket_fails( + self, + s3_bucket_with_object: S3Bucket, + ): + with pytest.raises(ClientError): + s3_bucket_with_object.move_object( + "object", "object_copy_1", to_bucket="nonexistent-bucket" + ) + assert s3_bucket_with_object.read_path("object") == b"TEST" + + @pytest.mark.parametrize("client_parameters", aws_clients[-1:], indirect=True) + def test_move_object_onto_itself_fails( + self, + s3_bucket_with_object: S3Bucket, + ): + with pytest.raises(ClientError): + s3_bucket_with_object.move_object("object", "object") + assert s3_bucket_with_object.read_path("object") == b"TEST" + + @pytest.mark.parametrize("client_parameters", aws_clients[-1:], indirect=True) + def test_move_object_between_buckets( + self, + s3_bucket_with_object: S3Bucket, + s3_bucket_2_empty: S3Bucket, + ): + s3_bucket_with_object.move_object( + "object", "object_copy_1", to_bucket=s3_bucket_2_empty + ) + assert s3_bucket_2_empty.read_path("object_copy_1") == b"TEST" + + with pytest.raises(ClientError): + assert s3_bucket_with_object.read_path("object") == b"TEST" From ce851e0901a98af54df51d18b80d45ba86b5951e Mon Sep 17 00:00:00 2001 From: markbruning <125313351+markbruning@users.noreply.github.com> Date: Wed, 11 Oct 2023 06:59:58 -0500 Subject: [PATCH 04/11] Fix s3 session creation in deployment steps push_to_s3 and pull_from_s3 (#322) Co-authored-by: Alexander Streed --- CHANGELOG.md | 1 + prefect_aws/deployments/steps.py | 68 ++++++++++++++++------ tests/deploments/test_steps.py | 96 +++++++++++++++++++++++++++++++- 3 files changed, 147 insertions(+), 18 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c73c15ff..a310bab3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,6 +17,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ### Changed - Changed `push_to_s3` deployment step function to write paths `as_posix()` to allow support for deploying from windows [#314](https://github.com/PrefectHQ/prefect-aws/pull/314) +- Changed `push_to_s3` and `pull_from_s3` deployment steps to properly create a boto3 session client if the passed credentials are a referenced AwsCredentials block [#322](https://github.com/PrefectHQ/prefect-aws/pull/322) ### Fixed diff --git a/prefect_aws/deployments/steps.py b/prefect_aws/deployments/steps.py index 5609a044..77930465 100644 --- a/prefect_aws/deployments/steps.py +++ b/prefect_aws/deployments/steps.py @@ -91,14 +91,7 @@ def push_to_s3( ``` """ - if credentials is None: - credentials = {} - if client_parameters is None: - client_parameters = {} - advanced_config = client_parameters.pop("config", {}) - client = boto3.client( - "s3", **credentials, **client_parameters, config=Config(**advanced_config) - ) + s3 = get_s3_client(credentials=credentials, client_parameters=client_parameters) local_path = Path.cwd() @@ -117,7 +110,7 @@ def push_to_s3( continue elif not local_file_path.is_dir(): remote_file_path = Path(folder) / local_file_path.relative_to(local_path) - client.upload_file( + s3.upload_file( str(local_file_path), bucket, str(remote_file_path.as_posix()) ) @@ -174,14 +167,7 @@ def pull_from_s3( credentials: "{{ prefect.blocks.aws-credentials.dev-credentials }}" ``` """ - if credentials is None: - credentials = {} - if client_parameters is None: - client_parameters = {} - advanced_config = client_parameters.pop("config", {}) - - session = boto3.Session(**credentials) - s3 = session.client("s3", **client_parameters, config=Config(**advanced_config)) + s3 = get_s3_client(credentials=credentials, client_parameters=client_parameters) local_path = Path.cwd() @@ -206,3 +192,51 @@ def pull_from_s3( "folder": folder, "directory": str(local_path), } + + +def get_s3_client( + credentials: Optional[Dict] = None, + client_parameters: Optional[Dict] = None, +) -> dict: + if credentials is None: + credentials = {} + if client_parameters is None: + client_parameters = {} + + # Get credentials from credentials (regardless if block or not) + aws_access_key_id = credentials.get("aws_access_key_id", None) + aws_secret_access_key = credentials.get("aws_secret_access_key", None) + aws_session_token = credentials.get("aws_session_token", None) + + # Get remaining session info from credentials, or client_parameters + profile_name = credentials.get( + "profile_name", client_parameters.get("profile_name", None) + ) + region_name = credentials.get( + "region_name", client_parameters.get("region_name", None) + ) + + # Get additional info from client_parameters, otherwise credentials input (if block) + aws_client_parameters = credentials.get("aws_client_parameters", client_parameters) + api_version = aws_client_parameters.get("api_version", None) + endpoint_url = aws_client_parameters.get("endpoint_url", None) + use_ssl = aws_client_parameters.get("use_ssl", None) + verify = aws_client_parameters.get("verify", None) + config_params = aws_client_parameters.get("config", {}) + config = Config(**config_params) + + session = boto3.Session( + aws_access_key_id=aws_access_key_id, + aws_secret_access_key=aws_secret_access_key, + aws_session_token=aws_session_token, + profile_name=profile_name, + region_name=region_name, + ) + return session.client( + "s3", + api_version=api_version, + endpoint_url=endpoint_url, + use_ssl=use_ssl, + verify=verify, + config=config, + ) diff --git a/tests/deploments/test_steps.py b/tests/deploments/test_steps.py index a2312d18..c78c3578 100644 --- a/tests/deploments/test_steps.py +++ b/tests/deploments/test_steps.py @@ -1,12 +1,14 @@ import os import sys from pathlib import Path, PurePath, PurePosixPath +from unittest.mock import patch import boto3 import pytest from moto import mock_s3 -from prefect_aws.deployments.steps import pull_from_s3, push_to_s3 +from prefect_aws import AwsCredentials +from prefect_aws.deployments.steps import get_s3_client, pull_from_s3, push_to_s3 @pytest.fixture @@ -173,6 +175,98 @@ def test_push_pull_empty_folders(s3_setup, tmp_path, mock_aws_credentials): assert not (tmp_path / "empty2_copy").exists() +@pytest.mark.skipif(sys.version_info < (3, 8), reason="requires Python 3.8+") +def test_s3_session_with_params(): + with patch("boto3.Session") as mock_session: + get_s3_client( + credentials={ + "aws_access_key_id": "THE_KEY", + "aws_secret_access_key": "SHHH!", + "profile_name": "foo", + "region_name": "us-weast-1", + "aws_client_parameters": { + "api_version": "v1", + "config": {"connect_timeout": 300}, + }, + } + ) + get_s3_client( + credentials={ + "aws_access_key_id": "THE_KEY", + "aws_secret_access_key": "SHHH!", + }, + client_parameters={ + "region_name": "us-west-1", + "config": {"signature_version": "s3v4"}, + }, + ) + creds_block = AwsCredentials( + aws_access_key_id="BlockKey", + aws_secret_access_key="BlockSecret", + aws_session_token="BlockToken", + profile_name="BlockProfile", + region_name="BlockRegion", + aws_client_parameters={ + "api_version": "v1", + "use_ssl": True, + "verify": True, + "endpoint_url": "BlockEndpoint", + "config": {"connect_timeout": 123}, + }, + ) + get_s3_client(credentials=creds_block.dict()) + all_calls = mock_session.mock_calls + assert len(all_calls) == 6 + assert all_calls[0].kwargs == { + "aws_access_key_id": "THE_KEY", + "aws_secret_access_key": "SHHH!", + "aws_session_token": None, + "profile_name": "foo", + "region_name": "us-weast-1", + } + assert all_calls[1].args[0] == "s3" + assert { + "api_version": "v1", + "endpoint_url": None, + "use_ssl": None, + "verify": None, + }.items() <= all_calls[1].kwargs.items() + assert all_calls[1].kwargs.get("config").connect_timeout == 300 + assert all_calls[1].kwargs.get("config").signature_version is None + assert all_calls[2].kwargs == { + "aws_access_key_id": "THE_KEY", + "aws_secret_access_key": "SHHH!", + "aws_session_token": None, + "profile_name": None, + "region_name": "us-west-1", + } + assert all_calls[3].args[0] == "s3" + assert { + "api_version": None, + "endpoint_url": None, + "use_ssl": None, + "verify": None, + }.items() <= all_calls[3].kwargs.items() + assert all_calls[3].kwargs.get("config").connect_timeout == 60 + assert all_calls[3].kwargs.get("config").signature_version == "s3v4" + assert all_calls[4].kwargs == { + "aws_access_key_id": "BlockKey", + "aws_secret_access_key": creds_block.aws_secret_access_key, + "aws_session_token": "BlockToken", + "profile_name": "BlockProfile", + "region_name": "BlockRegion", + } + assert all_calls[5].args[0] == "s3" + assert { + "api_version": "v1", + "use_ssl": True, + "verify": True, + "endpoint_url": "BlockEndpoint", + }.items() <= all_calls[5].kwargs.items() + assert all_calls[5].kwargs.get("config").connect_timeout == 123 + assert all_calls[5].kwargs.get("config").signature_version is None + + def test_custom_credentials_and_client_parameters(s3_setup, tmp_files): s3, bucket_name = s3_setup folder = "my-project" From c4939d76c8868c98a187fad47da4a09c1e11121f Mon Sep 17 00:00:00 2001 From: Alexander Streed Date: Fri, 13 Oct 2023 10:21:37 -0500 Subject: [PATCH 05/11] Prep for 0.4.1 release (#323) --- .github/workflows/tests.yml | 6 +-- CHANGELOG.md | 37 ++++++++++++++---- README.md | 18 ++++----- docs/img/favicon.ico | Bin 15406 -> 15406 bytes .../img/prefect-logo-mark-solid-white-500.png | Bin 16294 -> 0 bytes docs/img/prefect-logo-mark.png | Bin 0 -> 4240 bytes docs/img/prefect-logo-white.png | Bin 2214 -> 0 bytes docs/index.md | 18 ++++----- docs/stylesheets/extra.css | 14 +++---- mkdocs.yml | 2 +- prefect_aws/credentials.py | 4 +- prefect_aws/ecs.py | 2 +- prefect_aws/s3.py | 2 +- prefect_aws/secrets_manager.py | 2 +- prefect_aws/workers/ecs_worker.py | 2 +- requirements-dev.txt | 1 + tests/test_ecs.py | 4 +- tests/workers/test_ecs_worker.py | 2 +- 18 files changed, 65 insertions(+), 49 deletions(-) delete mode 100644 docs/img/prefect-logo-mark-solid-white-500.png create mode 100644 docs/img/prefect-logo-mark.png delete mode 100644 docs/img/prefect-logo-white.png diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index a52bfdeb..c7a7acea 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -9,10 +9,10 @@ jobs: strategy: matrix: python-version: - - "3.7" - "3.8" - "3.9" - "3.10" + - "3.11" fail-fast: false steps: - uses: actions/checkout@v4 @@ -33,9 +33,7 @@ jobs: env: PREFECT_SERVER_DATABASE_CONNECTION_URL: "sqlite+aiosqlite:///./collection-tests.db" run: | - prefect server database reset -y - coverage run --branch -m pytest tests -vv - coverage report + pytest --cov=prefect_aws --no-cov-on-fail --cov-report=term-missing --cov-branch tests -n auto -vv - name: Run mkdocs build run: | diff --git a/CHANGELOG.md b/CHANGELOG.md index a310bab3..ec431ba7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,25 +7,46 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## Unreleased -- AWS S3 copy and move tasks and `S3Bucket` methods - [#316](https://github.com/PrefectHQ/prefect-aws/pull/316) +### Added + +### Changed + +### Fixed + +### Deprecated + +### Removed + +## 0.4.1 + +Released October 13th, 2023. ### Added -- Added retries to ECS task run creation for ECS worker - [#303](https://github.com/PrefectHQ/prefect-aws/pull/303) -- Added support to `ECSWorker` for `awsvpcConfiguration` [#304](https://github.com/PrefectHQ/prefect-aws/pull/304) +- AWS S3 copy and move tasks and `S3Bucket` methods - [#316](https://github.com/PrefectHQ/prefect-aws/pull/316) + +### Fixed + +- `ECSWorker` issue where defining a custom network configuration with a subnet would erroneously report it as missing from the VPC when more than one subnet exists in the VPC. [#321](https://github.com/PrefectHQ/prefect-aws/pull/321) +- Updated `push_to_s3` and `pull_from_s3` deployment steps to properly create a boto3 session client if the passed credentials are a referenced `AwsCredentials` block [#322](https://github.com/PrefectHQ/prefect-aws/pull/322) + +## 0.4.0 + +Released October 5th, 2023. ### Changed - Changed `push_to_s3` deployment step function to write paths `as_posix()` to allow support for deploying from windows [#314](https://github.com/PrefectHQ/prefect-aws/pull/314) -- Changed `push_to_s3` and `pull_from_s3` deployment steps to properly create a boto3 session client if the passed credentials are a referenced AwsCredentials block [#322](https://github.com/PrefectHQ/prefect-aws/pull/322) +- Conditional imports to support operating with pydantic>2 installed - [#317](https://github.com/PrefectHQ/prefect-aws/pull/317) -### Fixed +## 0.3.7 -- Resolved an issue where defining a custom network configuration with a subnet would erroneously report it as missing from the VPC when more than one subnet exists in the VPC. [#321](https://github.com/PrefectHQ/prefect-aws/pull/321) +Released August 31st, 2023. -### Deprecated +### Added -### Removed +- Added retries to ECS task run creation for ECS worker - [#303](https://github.com/PrefectHQ/prefect-aws/pull/303) +- Added support to `ECSWorker` for `awsvpcConfiguration` [#304](https://github.com/PrefectHQ/prefect-aws/pull/304) ## 0.3.6 diff --git a/README.md b/README.md index 596765bd..952f8fd2 100644 --- a/README.md +++ b/README.md @@ -1,26 +1,24 @@ -# Incorporate AWS into your Prefect workflows with `prefect-aws` +# `prefect-aws`

- -
- PyPI + PyPI - + - + - +
- + - +

## Welcome! -The `prefect-aws` collection makes it easy to leverage the capabilities of AWS in your flows, featuring support for ECS, S3, Secrets Manager, Batch Job, and Client Waiter. +`prefect-aws` makes it easy to leverage the capabilities of AWS in your flows, featuring support for ECS, S3, Secrets Manager, and Batch. Visit the full docs [here](https://PrefectHQ.github.io/prefect-aws). diff --git a/docs/img/favicon.ico b/docs/img/favicon.ico index c4b421585b5f5cbbb793df9d0f0c7c09341d5989..159c41525dd92400685496c48aac0b6f45df2d68 100644 GIT binary patch literal 15406 zcmeHOdr(wW7{86p^p-awiM$da99|AcTKF6bZBole;*6R#Bh_R@GuWVsfdg8RUL*&lqJ-^H`uz1&^)-o3jnw#=M4d+y%ve82B^ z&VJ{7=X?mlUBW2A*H>V5gphr=AfyR`5D_uhj`b6S&)Bo1q?_Nz2tq@UAjGnFm? ze9kJxzv}9$vW*+o53oWi%K(@49eodJDO|5D*+BmX$lv!qn|*Vq#X#P{PtmHUHj_}9d zRjO~w4=8=-$TKEy`3-U`Dr!+y(J82S2tO0S$_!TEpKSg0_xFdGOr{F6+4Qf;WF&VK z=whV;x=|MvY@#tiqshz$79Z*k{Xqc_(yQ5P z=weH=MK51_+@b%C*RqG`b*KKEbuWAM-`sa`-ceV6e}a}TU1)bNDF5B@wwn1b=fAG% zG(Ec@Q(P+yElu=e^%;6@;XJX;`yZ~>qQ9}ehO(9}va%QYIVV4ys-UMcXFKE%)aZYH z$pU&|<#Gz}yMIXEQvDo9#qlFFo$ZtCaiB)Oy?&JR1IK~=d%gPY#9z+xAHw!5IczO3 zbzQT}KR*7hD1)+(X^|n9ekua;+Q?5&?T~xS9*ZH3~ zDTYcv-AVnXZoBrMWWW76GpE_rjo-7@`Ink9iM}W+9TLZ>@qNzd=e`K>^f!kNX!8Af z=;t`J)>`@3s!uNGDf$ruAr_M3b9Ear-(dX3<4+&A#Gc?C_9Fg7?8{?d9t-lA%#}5_ z8@g?s3;bW6`SVX-pZOp#-@YNUVCKJ5FMe(JLC%b|m;d28pOTtjxfgPVYb9A%@sInQ$w`k=$+j)j zd%c4mOG|QzZ_pb*;>e6?skE)=U21J=w8SwZ0sn;ParDs#TdA$Jnf^5OSjB;!iXVK( z$3;nOOty}U5XPVeY02BAImkg~`x@RXPC&ubB7JA{wdV8hq0)6@UK1xqb9CFtL z{8E3~@wj)qFPd;>u9kQHz8?6o?tZf4Lz>6tc%Ywh>@3I6ZE5&rtR0lZ1VQ6syu_Pr#4+ z2jrX1nm(0|mVZV4M*H`PQaf%pe%zZZUNoP|SRN^IXQkL2`Fy+ZV?Q=~##C{ShMX~Z z4y+^C@y_kQj~wS+J4&qbg&oB!f3NYUTGjhHXZ;WMk+_S8f6>Z2#%%xI-D1vBaG<)o zN@ww7ZNmOBB{7bQ*`C?dt(@;6=bE<>|IeM1LC9I;dkr zyFzD9e#dgU=4-+%)u%Up^bJ3hId=x_`nZJJFSJBv^r>rBWl>c4Sf{Z=mn-md zeh}|I!OlX<_U)mD+A47^iyW`~KMh<)ZS6|@QY={4q;0)Ey9>Wwc;)?g@hk6ZW2D|+pF literal 15406 zcmeHOX>e256~0c2jl9b%wk1pMk4}DM+RU^~NR!qfl;D)rq-h{iy3Hg_r<0_ehA^Q7 z8Xyi~Cln}wn1p>Z`%c&lW-(v`wj@fRkW6WrP9Qj7AwTK8)9+mAxss4&$(C%Vt!Cal zE$`m*opZl?&b{Yc36UV~7nzv?cdK~mYeGCMgs@t@pC2#^QHk#!ddT;ov^1M& zP6p9iDbck*V)D@TK13^_!vg3~k?N#pF?bQYr2La+5A7`=TAxR>B#S)L(ly5bafF77 zAGFtnY+rBj(7__2&CubaZ1T*>(60SRXjCq*J?Q6=w)#Q)1UhdyNi`;oc0z~Mx#XFj zsoffawj*F3zRHD4;|wdq_HnFm2z00g7c|e}EUjg@(3;T4XTMJJ+}_;BL-vvzZME0G z=P{?MoI&bRuMXQ(^b%})V$O+Y`~&Tc(G=S`)d+JyQ}_G7ailkIG#cKH*kk4U~_ zKV!x91TOYZ(Y$Dp=tb*tuhKH(SiH}Orfn>qrPGkVAy1L>wKD5pv@ml5Rfz0Z^c_hb zQvRlVh1S4leF&fP7x=BY88>NKN(H?w@+0Y^GtWr*TOfZO{0;lv#qiziduOD>|E4t2 z1j9^vU6?u}BV1c4e|sN=s^G6y!3LGE!9v(zHhk|?`0qcZ5WQizLx1QsC(<|&PG0bD zQvR+2Mf%INumOA~y$>6_2OCU-4JIQ7{V|#7wWJ&LqUaUezO$dD{Jn(=ZG*qv0QoC0 z2JrhdAM$5Heg))DfDOj>j-UMfkiQc#XCuY{K3~Q@<{`(wcQFQULH;;H{Nx`jQfN;h z(H7W%dAJNV;4zp78%&1{roaZ{<0!w{q|nEZ&vCm7@;P>Y03IxW4Q9ax?=TOO=<>921_{l$ER_L&aXfJHA)i(ygzFmdKQndG`@H^zY1PP~Tn^94#pns&HU zXf4CA{uw6r+BXL53wZt4>I;mdOij~{_Y;~{?zavr7dvT? z+zNyf5%xE%`Fc^KRiQH$q8iwM#~_3+P^PDA=YHt74%F9!!-WU7r~-~}g%#Ask5YfT zwT0>-{{-Y8g8V%g1AkwDn54|i&@K%OAz#N)E)?KcMT6zGrwwVLzb3n76xO{qjDbc+ zFb4a4zJUF)&lf0INB@J@(Ym~#yNdO-`f8$+HszkN>deLxV=Qhb*Urrb!2ip4i!94y9*lN3pB|?_yYGrqv|df z9+r7`n4tP6g?8ZiME8}|`HE`uTVHolek%^_KWS^ub5sR;uFBQ0pL1U>6Yfoq2yNwv zMEB)jxpG}E*VmoWUze}nc^^z`b4J}=WIVU*nIz@Xs(g2VzPeof9wYsA`FtmP;iF`; z>t>K?NR8f%hm54(4Wp1eUmz> zuP(RsZv7Q0`{A&{e>CpfM?kESU;w}nmZI1f%!gin1@DyKZ7H*^sA zH{?07*Ywb`?8{UMJ(gtjL#*xn6@V`!{*A!TdsSYeFU|JQV&mtuFntrvH1vf(h$!FO zUAjvAoAaG=ZO1tm-sf=6f%7i&((ll$)a6vs`%#QXdUs(ACs*R%0{pyBIDhc|)$4Fl&fKck;_ub-9p_Z2#2 zo`LuI%puM@a^86F^9NTtj7X5kKumr82Gz+okz@{8sO)AE9X0z zLp+b~Sx@-I0_Ko67W54KCySjjXU?^L&fDu8;#jcKFBWhOgVz&0kG&k~8Td~D|1r+@ zVhp(U!egMv0$$VUv4CqJdMuchM(UK*YqtgooOtQJm*mfBi&NGNbPjRdfH|bEC)$Vw zEfZ5b$DR;pYfHsXoBDTMo!Q@75`P`=pG3`p>n@ygXAbGHKwnP;!~*zIaLDymidtPN z)U!{DCS{n|i1Q>PsZ`|n_aE=~68~A?KMnlMp~Jwx-^Zb?h-ESsfJ4jSg97&j+*ZmH1LcOU0__=1ob*ma5hwh05SkLM6*cS_26=`ZsKcQZKItag4 z2R^5C9b-~SLqr*Pl^PE&N&L1FC!MnrorVrv`)hX|!~R&)xu&M6sKcqZ2K(x8L3JSD zXB|}Na1+;7T;Gnh4w66il4gtnQ61OKeDgRYhkUU>t|yG-dN)n2eNw1*hqi$~SRbAr z!^I-j1LF~s{EVzI*-G}H?&zemz<-K4ZU_=dr#osPyg&T+`Fl`u;-o437}M zmscu{O!$Omq`Vrdv7km%7Bv?C82D}QLz=8zN)Evfas5)p0^t8oj!2%Zam`3qFAfxH zz}yYCx$iSSgPNPF{4Vh}ZO^|=^~Kd#JH<4&MD-b&R9Ey0oiNoQ7BtHmc$iq=nw_Dd z4jK|a<3$~mno+0xVnl*^?WIKJ=BnJgYNM6U{NW59&Z5N_KaVHoa{`^nFQKCaFVmsI zLx=^g_55!Q%9X`AO7jV`J23Yi zAN)Ee**0`6J&X3`K0~|v)MHJj2JZ{p^D^Da2-LcPpK-fTi?#I^wBox$+wfds%craJ zT3mI-K6?k(ZsLny*F%oUR*_8G@?N0Ld9}1Y_YUTsXrW@-R;VYD(RQU+TbTP&XOPv?2R9;Lt%@k==ZEzR|s|uW--%PJ+5-w84FT zL5_0gcrZ_baqFME&i(;o!SiDsc(o#9MAN*?x;3McTz^{yo}97pc>$a&^{e0aAo~aC z;K)v+hJwe`bLMT*pM=G!+dKH)_4cR6+*Gm`4?;b$64y1@S`E+fV9frXyAHps`!yDz zE?m3=xNk+y5r4eF@8gf3m?w_H@8L|(8sK)t2Dh$9{P2h1Nzj>f9YbgCvBxiCFZ0A^ z*$#VufwLl=;CFkv#STC16iL(oo;Z9w>5xBZ4{y-#e5OA}>mStrAB4>t;%jdoeid=7_B8#8?WpJ84_?u+a=1M7efvwIpoF NR+s-j|KD$ce*-+RrwsrA diff --git a/docs/img/prefect-logo-mark-solid-white-500.png b/docs/img/prefect-logo-mark-solid-white-500.png deleted file mode 100644 index f83aa6ef6a34ee4c596bd1c7c2046a2f05cb9342..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16294 zcmeHui93{U-}jhCO)18bC1MPr#S%*PVJu}=wkQgrjY9TqO7<{Gq6|{fC=!yag=j;I z?35&z*MTDY)qN19Ds@i@vLgtHu7KVvdv|0_|Io}kKi0N zJq^D$Upjo;2mWec{jjC(g9Y!F5u!N)+~fZfEs+f zc*gfMHsIm~Pam}aP2BnjHTcYWERVyk5Ai*(iL*1cz!JQ?&tMPADaa|{w6&-U$}1|$!U$QPKu_P(0kWPx z5`P!@*E)J8L+<(t zre3aR-2(Ma`<~Ha=~z}lRd&Ck)xUK7clv)B^IvAOs8TzA^o)(Qks16%w&UbL=oI}*+^^EZ*zUu4)7d9+->mfbbgt)9wDR+@ zJbS{e#(4F%9?3tCRyN&`S3LdeZBplSkD1A{!@B#AoOOOEb#{LqbyrHB_>ItChZ0>} zc$)$j0;ej5=aw}6zfT9WeC#h_lAD4;ZkJBNCt7-$R;}uiPx$%oWYyn%TTs|!4Ps_y z<~2$*!EkPQf#UvnwRr4uyL`dw?J=(Q@2U@eAFEeP!bBiYSOgpTqaI#YE8AXRVbosB zkgn5SIKx&=>~xP4s=WSp0F662xl(V{;?rQ$Tu6a^|M%Q9bE^+w0kwOa2Xf!@qVWhM zCZbLcCHbQUR}|kf!`bM}#DuJby3@1;Uu6$(ZQLZNq01VBKt)Mn-+AuwzUb_9Hm*-7 z_!uoZingie>fe3%j^M9eH1p#aZoW$xrzZqwRR~nI-7Z#X45MhZQY&ytc(rY8tlfmU zlkt^`GmEa}L#os4{gZSXC719lE&581M9^eTfFc7CY512aya3zg5@Rai47I!Q%hk?G zWW)Ghwu$}*1<;SP^P4_*54QvJbifsvoX)~z-xoYn%h?w@zg>27@3 zm9>_ql_|YZ%x9pXZn(q0G~~?(J~aN69TFXCN51g%67uHl-y!)5q>kL0Ty~BgkvHL1 zp6x#ID{`7RKSDPC6bc5S@-IBsK*|a&2;Kj)qy3)J%o$?;mo1`1^ng-r!SJvI<4k9O z7;>B&9qut6nqjV!eF+&;)xq{7CxELwdP=yVpF?=B&Zz^)pIjHp%!?1Awn*2tExz9r z5j~h$EWt?qJk5rllZa?ry6h~Artx^ZYc{?^JLo(2t}|NdJp+qHqBH9eNKJGyi5XGa zy~mYSFztL1y>C}UTN=*!`pwOyu2d60VMYX=RzKrZL-HHtVv)5{g zohq;2yZcB+wD~0L3AH!M-X*cNcd>1pvN+Mc@t*jQ?^8O(;fPjgY-WPyJBu$P#c1qY zar5&&na%q!3-bA|2W$@PAQ=i(R~sCN9(>oyZ2R-T6{w?*2z+DS&Y^o*Go1?Jn!jpr zb}RZdklps3rQ=bUMLf34<-bGK z@x&C7Q*eQA@wHw46ryO{;^;K5Q=Nv+fdBhSllEk-raEok>h{= z=Z)Q)lS$@B?uOE_5uL$!=hT;=h?j$wC(hgJwW!VvCSJ`~?=_^6_t!ohD9eg@C9NzM zj6`cLVVb&Lvq%6rd$b=xUUUwx?0<#F)ixE3+!;b1kUl;=WdaH@`C;65(bxc1I?5HL zywt~6nKqDPFQ6_IoKb9A(~j0$z~p^qQ=CyA)Y}`(J0jty$mlDd<=ZHZB?Z4+ zPIg;X8)qa8D3qERXrWIZyeSCdq+QJSe3kOT!MW#7(pox0j~tI0G~^&S3LA_B zHv0VOKg{y(dwYZ<>qOUy8>nwg(FZx>)Yxk~Zj!>|FICyJ3q=0(r?QI)`-bu%Pde8g z7Fw+CcR9cZX~Oc)aIs}1*-&KDxPE-IdB@w+EZ5p{K_#hA=@J{t(F zXME6_X@@;Kx%6tp@mie3)0>qWYaV_Ycwn#H)MByx>|@Hn4qbDt$h%h%B(Jwg=7dwO zNK^0|PuCy2y+m^!iBxavPHa|*XfhO6&?L$gtF#se+f+R?-<^!mmESiaiwh@Xs_f}$ z)~be%ozr;#6p6N7j3En+l;N8k6GPh!M(!$^ICJd8tsOtGnr2m7GKNa&yBQ;OBeHJy zMdwkBL)Z>IS?jX0<7r85<`@-z$ zbc#FWRWd1yT33r{zJWhRJ1L}|GVC2+wRo*fY4f@G=1kIB0;QD{Zr*MhAHqCZ*sjQh ziNKS=JTiS~LXBcM(xM;vNe-QBHgs*u>J3V;D|I_s6l)>K*5GJS#~pP4wI_Fl%fgh^ z%4~^%Z68je2>%B`e6tzLKyo4?$)}Qc&DLkUK6iYIUv_P;=4FG`=P5-rvV^-w_0`uK zP}paN*}MJ1lN16yxy-xGF+sZ1DKA4Edxxu*4MzvkS2;AREa|@;ZNw$C-x0&b5yVE2 z$W(?#lPaT8BsWMP$8dWh<85io@}tr`{nhGrX-uHHnj;i8{Jb8#OYMs>;ggJPZOW9G zoJGl<)5=r0IS<|9TJc4K8r7B_T^4-NO}5r3-s@=Sbtl`dqgS^2cg%7?K1CqUV?5$k z&R+brqg^5-2?+?xNPz%7D(kaD{cEeYk<>-3J={{exRn{X)Yh$Wr$TNDrq#Ox~!&LyyLiP8%n! z9W{M*y#07ZxbdVB5lR}&24Y@kJyo?eF8uI-Q>EEwpCZHbI|$t@!GNC0Wy2rZi0vPi zJlh2d>_@g1b0FD~n3%-67xq-AfymELbCKb29Z1T7y^dL_2ExeDH*eG zC`Xq0WD`5vwm^NN%~n~j=r1*Sn(@_2{^rN>(NN)zP14E#!XX;4_2yYx<&nNxb!Nj$?Qf65b=wKYMZY}%=3m73$Z%b_cAF!) zox;N{-LMgD?h*ZE#_xJddS&U%vzfJz7o{n;UzPYb$e}^1*dV04-oP6Z;g~iFjtjf4 zI5Fs+?P6B5t7T)u*5l61t;g2a6Wsh~MfsQ&D_sao*x(S8drC;V;Ks3t!pt7HzYo>s%C}l zY>`^qM-MP1Na_wuYxjK&7mC-{JjeU?0>RCl+)Ekv76e(Q@(rpX6iy~Km{u*v_?6n3 zS-ci(Js?TUnMXMv;T4B16QJHn()Wv;coEYlwDh|>RmpEHY(>X9J~JmkC!1B`%fv#d zpwKPr9ByB9GOuuc#~=P+B^P%Yt=EUXy0ToN0D~VXXp6hh$L{RqMoTPlZ0n2tG#Sy!bulu$}>=b}54^5glJ#;v)74W{$mT+G3wL6o`oTKPNIexoiaT zkKZa@(xRD=KZyUCV*~2rASlc9@+`eLw}L!yr8sz|SAtT*Rhh)$*0^r85%@i@*&fo4 zXPIRQnkSBacV<#m*Lo+kABP=JJwAOE&+>)jEo_lwqXz~a?(vJw^3q3 zQIi*VSY41HBq~8G-X1r2pVJXW=zG@ru9G)j`V)kl_rT_dgy8c<=bgDyhW9a&p%}I~ zSA_zMVNIaz1zU4@jYM@-qivBqn=cGOM${SgW9)XHjA}0lJ{MJ#sDX~-i-SkG5 zQb$3lqv_J+P3)|YeKTHa8+0JvP$v7_prKbNXJ|i%Jx|ZviymAn8FQg5_g+LX{t~GO zrH7Ds|A1k(6hk4fg)r^)J+7rQ+;sb2_l>B#1zg-JRqC<*V6kjV@2(ij1Dzeyr~MUz z0y`KT`|pp;&o_(1w-vjK`o7x4hSoDgvsAU;qQ-)Wg#Zdsw=LrpDD<`p^Ff;NL49|k){rzAPr!70%MFjoNt~{rO~77em@;^c_)n$qNR~i+2-PL4Q*OKV0+J<(K6WUUE*!Wni@Y z%y*VUCToL%geRnop9TZC-w`sK=JqxWU$oP<&#W*-S&*g8IuLFhzN_~V%DzEV(J#JQ z7Ivur>23^MIRjVDYoT9;+8||}L0o%xJ3{Ab1X0X5#(OCA0ymh7Vv=g# zH3W(Qye_5xF}jaZYpC&<5vg@CSN^UnjKp-BBXXwP-|UU$9l->L&h)CZ>pLC2Jy0wy zdOm}T4Xt5}$kG0A)jgft4#$Z9*4Jb`-VrD5(n=G|R$q@2^1HjQGs^U7;ot$s4}}mj zUV#&EXc~+hM0o=tJ~6n^5&dj^TGF?Mu$+TmK6X~joREM11jUAKz$0>+Fznq5yMfcS zxE-HAcCIjg?rq)Ar+qfEQwYidPPWJz&I4L0>?~36W$F8lqEPRlnfws1s_BUVf?luy zfQYU^;if8wI|B+aT6NmrY2_olHn+TtLeSPSf;ZGi2Rf)+ZWjYAtTvU&-1J7$GvVsy8)XOEhQ_X`l(H}D>4 z|8!5Gfo5&P9!A;VIIH+a|G2OLlnDVr6@yjaA@Ub%Kj^sHDc|mSS4f>OEzx-OlWsQj z`_+*$Uz?6^_KjK5&bR!O0A*c?L0l#Qdl4*YV*c}DB-%Gd)0IZg31}c&oH)A&SitO~|Lym(vjhFmN!F0ydZEak&9`1@jAXngF*gZQAWD55P1`Jgy9-)2}p8@kTF6Xwh z|D#?Qed$D-KH3-h&XcYIJCP&|*lCYN)fzcnL*PiS$3|41R7Be&sg_mjGbykWfCOG7 zWbSQ!`Yt0$2dfWkHRgXFx(^(K?=jiWB|=?%KS@x{38K5B(SuQUKajtuBIDBUim%@; zgWgWgjbi@gE?ZDHanJBHc}v6o2aDU=7cV-z>r9DLxN#k?4Wbm2{(codhuY_Nz}xV5 zvT(ugP4qvO?XCHaj~8F=aTY*rL9vn@3oqD{7$F%(p@@W?GNwn8YmMl?kkdNtX(MGF zinvMCR=CBR0rO^`AIR%-KdZAI$8`*no4vLt*xKO9pT}F%zKPjrIusqJ+2(8I z#&xn2kwg&?p~(HiVsR3uNR?lCN|j~~lDtMQu1WRn`sl|DSU&x9O~1Wkl#Ycq^rXr1 z24X2VN$D@caSoVJ@V<#dKTOwd-*TVN=ysfO-x=d9ZM)bNViY*Ax#FM`S`u1nX`zEV zPVOL*asD>_d<$AuH|JlBO=olx-mV^f`q5%p%ee|cR0c9w4KK&8wqsOriGiI_Q}O3w zJ2m%`bLMLlj&mWoA(nEZh*J1$Gd$0fQrD-hfTDz|nu!;~JDs^o01oNNp5B4Nfv_aD zZZX}H7NO+Lbi8>|x$EN7gb<;2Rvh#cI@BMHmjnSx60?QQ@Y-GWsMhMbcZspM1qH2Sn}yes(?_H$udi^@J6Yzsjs4C9{uBWMFd-sD1R07xB*6^2IY>?E zTP*cTy`sii!5aHq4rKuG&M@M=lsE=GCt{!VTO`oq*Q+F1=OT0Rtc8Xx*+D=q>ss9o zh!Z>7=FzCz+;*1Ky(cklT{VxwYV~Pkjp3YtLJgHG3Y|`$fzi)DAI%bYM|RK1NL6m@ zV{#S;Lxuy(-nK1sLMPh*k<)SN4|pObxgmv2dtsI(5b8$ktncf~AWZkfzi0}E6!UK@ z-OJ`q4I}Uvhdp;Vzey$FGO_cnP-Sg&NI^&DHb3ZTEi}Lzmc^t>K`};O#Ym+!6rvm_9X3>T$qc75zo-Ng!&0d#->Fb$5v=s`Y z7L7y&VJ^KGYnOcZB>H=MYu^+~bM;EQO1_p;s8%86QEnXc<(i2E8)6*}B+I?Ce{V(v z%4X2DJULd|{I#|@-DmJp;<;lq;c$Onh({dX=BpM@TbIZBusj9k2(MYnbsw72!CE7E z>;Q2x4tdj{+^P|7PcM2bYN@$Lsy06;D783Alh3QT)`Y&px>Xh-T*f#BhR4J1cp^oVhPN$xmdyrubCvr|sHkiD0Ju*YRq{?@km(`oV-T*{>% zK=Sx$N)&ND^^d)mL*G$L%qrZ&xKSHFC-(B9HCJOwF_(zR!L*IU8*4FNMx>s8%&dz8 zyf14qVq`BvyhuX;YAGu1dA*X<*{Lgev*o>5jVunkXdm-udD)TuKlM zlr}~56&ciZ-L1K|WadYp7W~Zgo=|NNg&gjq5;gcJ;owq6B-9TmCH1aIyaZqvp}cpd zL;b9ay_Pc{u3@6y`79@D*8C#!U&MpoByMATG8BJy3n8>{RIlPmn=OrCMs|_WO3RH7 zqk-N<(3U)ZlJkX!e6qs6&DiNbRZouONe#wjhzyv1Qv$i^Cu72ueSPS#qMR zxL;y^e|gtWug7JkHxIM0`~YnF!OO1W*dGbKt>>NIxKi0Tli~rO=qtatb1!s>VU~IU z@;e~)W?0<%j0qi`H1$jQSk~-Xu87kPkgsH=+v4cK(6|w;_L6TUX%xrYv3QXMZOSGq z&B4-KBVwH^?BJUJp^x<7q?rBid(YpZTU_QeVpbv#dd8Xo@|QaNHb$zaHwM^5i-|cJ z;1%7U_b48&(nhhZ`F`%2egSBs3uhywjhN=*8?PZhy|#FsgU=?Bl)7XieqP>!gxW1m zb1Q2^oXoSN<1pg*v*C)bSZ`+HJI}zkhjJXR#D;!M2>nd0IN#W^_U?yBu7_O;;?%bEtV1V|PfT=y_M1SDIvtYk9Ma-mLxRkE(%_76)#& z^HqlTzqJTC|1#k6c8HMM?XMU0G&iTyZ04tv1DJ%jTEAH^&A^udQsd+rW)lRteL`{O z%Mo!o3uw(K8OgSd;cOM@i#%IY8h*U@+}v_uyKkx_w>QL3aT9seGS;))R)?plwDAaC2{xBFMP%k3#nH%XYh~mUL*J z-uaxC3ENjJ)-fQHGQ355e$h`Xhq)`YD@apcw1qUqS??YdF^NC56M2#_%^dL>8@9@{ zjCQ&z^8LBj!&jD{8Dqnp=^EwSpQ^MI+GZ1{e1eAM1=_U;65+U+U9$Z&o+)D{!Ym=ewr2EwpHi`B+U@T7p%!N=YiP)aRI` z(PVl`YU{E{rS6E2h1jL6n|`6s?<&n;X9hYv!|-Kt-ikA9T_wBGPw-e*+SbM*#0_g( z=+A6_ZU5BGUvbXCAh9_&mS-EWP7A5SLMYtN-Pj#HIL?AF7}upI1fT()J|A;0L8Ey7 z^9iUMhIYhpY5NmsLZWeKO;YxYnB}aIGWGW5WKyt<{5^=VnxvOsJ`QgFm|DDv)g0#% zFgb>zAC0Kdl&maOiSp*P!=GwQ>Csm8s@~!IvEBe?VReu`X8;;9{&Q;doTBa&PHA>L zwYgq(hSz@8@BnE|^FUHU$dF)hFgk(7gb7)g&2#du-UvzmB~PHH#M-8yiTZOxo1h$* zmPp#?r@RTdfd|d1IU_852KaQ7B;RZ!sY|SJx34`OS|N@N*lvS7l_cq=ya6DDJLhGz z?g5a|i_#H2Cx8x>xq05x@C&0lkzn9ST6;FO);6?nR?L#+07U#x)X@9RYn@9bet zv;JZ^t%4h3d%`7aI*RpIH&DbZd{!;!Vw8;}bpsPWUQiSEQd}(;XMtI0V)YbNHM%+N z&clz8=vxtemA9(=OT0B}4%3-)H|4)B=2EoWhtkJSd#pr|kFoHxZ<`|%LeCY;zwcb> z`>l)gzFALQ!_B=TW%Zs;O7bdLmA6U5i;qIp$E||_kFI|)=kr&6f#w*)f<@41xw&s&v^N{m1yuD!Z(OCzR_y=?jIJNK~AXUsGTGj2x5V zm>-lfIL6da8)3fS-?_=L$m=sBB!RZt^g8SF%xE_wB~-gLsGb{35KH_`aai0PTIY1L z$tufseLsL-x=|TIOYx--$E~JrdxXE8Nxjh23{j)1Jn3Tx=YX4--pJkQ9_m-^6QOrz zP48MzOQ2>JW(zOxkZv{IhE8elbx>BYoOlfh=~rLWe)}f&6tmaQC%wpuTuI(%^s~L?d zUV2ngC~r%DGM{CAMci2Xop@|``sqx)J06m(5titA$}Kb6^4H^O1C}dS&J1TUu8OYQ z^)tdjKY=6OPha0e@)SvZiIX#+mh30QD&)z+g@>vfHxu{(BJeYPy4potjY5*gH0ZZ zdrMUd85ywWxbvC30^RT#!#0)rgBr2RnNm1Xut6nzZLqvgt--q_gl?;R-W(dh;pm8x z3mB2L-8XO2Dig>5J*+t{P^DC`q~FU#IdrQ z%)*zl-m+fSiXF#{CPAE&+DQKKvZbVja=A$-LE_sqbbq~2aov64SlZG~IQ za4<9Q+`4;C70ksen3^?NB)Si z-cN5n^p)XX+)AqKeK0m~b6}jEwTl*TDrWR#154X>NO1{fqpGv9y#pU=q~XkeTjZC!2;He0aobr(5Eu07 zaLfbk^~}ks@rtPk#%8ZFDeodfV%>Yx8)DharUk%PVHjL9;9J;4-yOdkzVtPwN`PQ9 zrh~{~Ml|eUiRC*Mqd4(LbE&rUhR-fK%9u&7P=(Sg)>{_mJ)6DN5G!)2{vt^h)Qyoz!5^b)sd$o>PE_bm#$MXQ zU2)dpEj0B_+3ydEW5n3p6K-8b+fi#c?C+FKc8!tYzj0Giu;vz&0ia}tuiqPpV|KE& z7Upd=uJeBJQPw+)YwgL{-Cf>mRi!*uZm zSX%uEld|~UQ{{T$5imy@v;aS4@joy7LhSoT|jVtbfcqpd1bA#QOWSjgSXz738(aN-!xdxJlHvLx0r#vW4AjHyl?5;zr< zkuO+-uXv6QgjN&O7?fLLw$-}y7`@dyJq>NIc3U_D{^)XJ>jY4c$I{%_?rCd^fH9S; zk~_88+#_8*TL`_&2pWU;&)Q_bqL4^a(vq}W2gMP4IsDy{AvF(wHl0;KMFA2LjeF+@QA2#Qvi@rF!LXdMcAZ{v39b)>$$J<7jk~&`sjP@x zYhjy|@3{N2_HWM=(p@+;UIG=Zm;}J_F#zK)FULimC=Lu)AbKsY7zh^s`!6l$H_|S?`y97@~bl0_i85%p% z7}~mB5dm<@A$SWkcbk5I0R~ZkWksW1tLsDTSvrOL*O(OeTakUZ~D??{8LaESugCHgASN0v2F-@)uF9jNEEis ztj_kr?C;~(n!_C+iD}P{vEm~}FcG0Embl9SoWcz3Y~^dopOr!3y!oK<8ln+q;l=ID z3aH75M0BL~OX9&MH~4&YGej9xqj|?^S(DJc#|FN9!!gIzS+;-?B;)7ng~L8v;D&&8 zon?;&zxWq2H(KykbmT$^mH-7x5YLogWFjJ_jzEC=8BFB_qt>R@7(;w4Y}^pji-%>> z@$&?cK$Qz%*Ud4ojF{!XzL}n0Lp`$PgHfZSU{FVY-Y>@7xgOCPUuj2OT?x2T7M2DT z!_?Q@_2Brf+4LInpL59)L-YfKY&mbP+;IE$lJ5h*5)V`PS96T0{8}s>6Cp0KsZ~be zaj1R6mlAt%wTARhJWTbz)u?7wb59vQ^hY>XX+>w9Bw(G9{DFe@wdSp@w$+x3%<zGj>tH;PC>##clIGCKEU3|^?}_cHpRB}n%>F8L%mV>T0~9sgaP>AG zyAk+_Y$PW!K`xbefY%3(s%if9jik$&(wqgPw8zNWC zNDzJYpDPtYO7NhMGUev(C4}gGzpw*I0E~w4#nl1R92xPDe8hHSBPve(2ZzqzBYa36 zhws5mjR<8(=8$i&r{Ea(7d&QtP6oQkI+#%~5lCrZ);?Q-PSz_$FfS!E?j&mzyu$h> zvfi^I%|=kbMx6&7>kA;*B3t&aPX%Q~KqZ5X@r7e<)+{*h+4y%=q!2JJgd)lPvo=Pp z>F=s|>;?##*RWsM|E`VD9Txm&LsuZi9i#W^pS7Xf*zk8l;#P2-tyd2V{IfPX)8&7e z`CrYj4CsH&?0I>$Pg{&0{fv6izVr+5YjfL`W9=%B zU)?*j0Cy}S@QJW=qUA<|8(UbD&gd+J!P)P;<1N=49&^W|8C=4Na@7(T{H?OZw$l zi!E%%-Rb4KJkys>EL@?a9sc3;B{#a8Q~_BgSrxVlj2LCz0)_rCD5z}AVb@G_-}u3quG(ypm&iI<%Azi#u*?P&^WlML&NB*CaRX-)kc8Zbu+ zdYT2pGA4)Q3^U3GJA=OpU;H@54R=IhIHz<@i6FI9fPxx~$-HnC0qU=wxslWKM+?kMu-2iMWKCy?v5>eki&UajN^4=x{HxX!Z>T}1~7v6S{PGv^8+2Vz_`I#NE> zac>C^TteZ8qf`FUwf;9dTURTg{_!9CjDt&hV;pY+POjhfONCUKdZ}0uWrM@$c>=V( zR>%TYn71Xp*Cb8jD|rSo8z(!!&=4pa=#Ol7y4&B3twyLGrnY=5FH3-i9E>jk z5agt&+uPhtpiX}gOp4CA5*BoOnuzbfVtAq4I;QMlOhbkA_bB>*pU9!h7dVbmwpIno zdnuF7H}g>*=wP*#W~-a!ZL1A%!;aL^x2d-Y*u7AC6kPg}PK1hfb#um= z*}j=e@-ySRW*(J_w2*uPZJ-Q?))uO(l0o-j5^EJx4NAA4yth;P4Ie6LBgsuCTNw(r z(wMIU`lvU7Q-AzlS2qiM9)z#&VD}(@#_y#Vd=^BTnAG98{8eTPD+CrA*iQL_i2fR_66zYn++`{_@^gV#6YVvC*4Cqu>_G z8v*-_r;pk(Bj(s%g~pQh`pe-g<%==ge63ZC`%~X}uFLo)4pY;4^PKXH>vdaDSDQnh zEL~NQs}ZQsUu|+k<2s%z`8yqo0x{#Eomy{Oe~@9A>@?PUQoo;9v;ca(Q6OftF@axR zs!eLfVUBxJzvhnc6qtT2cv0-8TXiT4L=`R6!TRwfG06@68j#QR6}@`v8%eg>kG z`l|nCicUdV_j#$s{V6 zw3esn&7tK=b+DI`$Wm0K0ey$B;^${f?iVX7sw4T!l)Td}?E?{{Xv_<+Xg7k$F=8a< zdqr&Yv$R_MY+{UcEQ5F>0VOw)9oz_csOrEWVlrvT-Th~0C83qs7k{Ei zeCG9Zi4N$^9r@Gvz_8Yr@&&NqmS%yz_};EX@iQZiNj@yp9!l(IaFeKJi|qH4sdx+o zvR`9^!K7f16*GC|NqteNw+phYC;=TN)Y?UecciBJxX}*-YS7 z57^MlR_Zeag;%U5jNO3bL1n&becv3KaGB%Sx{qN5p|=s^_I!KSGp?73t2Lt^ja_DA z3jdAk5p{s=Ec!n2={$Cf`qPo>mD()kw`rF*SB2O4bfIxoy?a_f$UBJt?;_8Ak2|RF zDnMk>;+=2ycMzxuck`>IKxnFN_*C62!6o_$E-U_XdDgZFOyhQM({=q-RqKKHWvSTZ zJE8=GmMh=4U0e4lxH+PE|J29MTM9&35w0vt2RR#hb+aZFP&VTF0B7V+4vj&nFQwd| zJi5c3u%jOYjttiBWE}ql-q_j^VeViB)0pGumIC>#wq@*meape}DHkhT^f}KuFH1i)~hG4?d>!>Q=4c zA^&g%-c4<;5U4o4-Xj7Y9`ahcJ0znz0{?P0-39Qy$||RJCeGi@6Yg{S)2Ev28xGa# zvn&chY=E57fc&0Yp+mhLfLVAN^baU`;pG37Pq4{yP!@R7GGx7YE8Gi$zrS)XQqk{U z)&SHQ)=5?XIfwt*H)v{fjN0-t{1|Mk26HF(|Ql!&Xri z1P%7=!<&3(E&ZHrg`roK8Qnowoi0N9&lgyou~^k9v&gT%k{lXn>$8jKl$jM5HJR0Sz{!Bq7Lv5Q=~zO^QmDu2d-@iBc>Wfh#xx zL_ZvzrA)?W@l)U;=hYS z5F~lpkYWx&BF*3$L~I2a%+TjyAY#6Tw*C+#f!(}BptOJCKvKltoTd*IevthR1~7L$ z6FmqjNs?H1fz$5xMr1?SXA?l;Eesl71 zoYcFag}XG7nibsp&A@&vT`+Q9&Cx#m0$`>0K}hD8SM`WhHn=-&*r@ z#z5s)tX$mecRC_zCp2DMyVU%mQ{lXHZpHP8cQ|jv;Gig^?OQIty+R;@hdoN_ zbh#mHKch9>H5L_B$J_Qfrd5oU;IKwPMxMOh@xnl4`~~k^XvlD;4!U_q zo?r|P31+%q_U2e8o<-KLwP{;fY1={a%`Jk4_%5yZ`69br+j$e|aL7-#XSsg^+p08- zBhgfD*=~_H(N-StK4o905hjemf+u`fI=XoCvU?OaN^ z7&AKGX0H#KTLmtz#mt|_!{lwOw(nS)9*C@hbc^*wqI{hZx+jS+`3#&W$`mQWRANAU zEEyVN;h{M`Dk|jv)Bo(T6f1Ovjp`l?`C1OQsrv=DsoJa6ZCwjUKxXL$-x<$8%5vaB7P=#!oiv9XUH ztBoN)(A7Nh`1$VZf5}BWEz`pekK)SR}UKI3_0djDE&z{Cs z3i>g)Wn3ZESCbz5hKcuVTSRzQ>}i?X)iUreX=*nEr~lmnrX^FqusA+$G}m{H9($09 zpR4FmXs1~x#@ynDEVcNLOr=_j`=;VB2lhp$AEDPwABa3zCr%oYJcK0=y$=mIXZ;{l zTn4VdYeykxH6!|KgZ~=1_t?hI~BFYx17?gHR(m7XIO1%k7sh;MqUH_jeJx z&u|UpnD_#v=&$nZfK*(^IYPHdr$IZ_*>IBSsAD5T<$&2c)u*{d`3c+u{N%H!#)xMm zb_JGv|Lu60>S#vCS?Aa0Oh*zu+l!wZhtVTps6F~fTLk*wS17k#vE)%BSZZuJUG*kT zL%LI?M7q_hW( zr3;D@sz2zAcEgD-=MT)(oX^l+wYxUeuJoP_yKaK~a$8w6T!}H&b6qMkY&~1#t!@z$ zkM+>+hLv2)w>zf-T~J~e7qEsdCaq7dT%k+dIOJazt?gk?^W4Kpa_+O-ZXa9>1+Z`$ znp{Lc3QofxNCHkn3rHMJBY2W7oAg>RzsN3O9Ec#A?QN{)WMt77&Z-P-;IlX}ZUag~- zl)9<2VXln~{C$kr;eb2Oa~o8_OI2hys50@j^6UnCAOv>9VSciGDM|c6EUv?&(?I0K z;3asiyO&fkmZ-o$`!odclaF!{o{9`UhU)kbhcT5BA;u;#3IE2ByZJrJuYu1cJ+kEE zaZTIv6QVaXZ)^>E(`PZRMvN6>5q^TzeUE|`kdaPF6m+#I(#ZgP=m%I(M31aTp|S;0 z$GWRX6PreF^L9q~f1=Dici6~9`?UEvBP1MusFbAxJg^95cX3UaeA-+`POwCA*#8=U zD*~6bW>jh3HGk?#*RBeBN8Y?Lz5fTN|6uXmU+{{nv(LKj7xiB~zdEGM%*_-ou5cvE zzP>$iVC2)AvGs6Qg7fRdQ^L59DMgXPiF0Zk3)5e`{1p!A;3vxLPYL&Td1{$z>MBH+ zF0hWnKRz6GAmwI8o|d{Cw>%IUxxPF%b+bt7!F`-Y$!H)Sa7)z!+xX1geiwDNZ+u~g z_!O|=5xUDRO7b-ndC6GH7KV1o;3ub|siS%@TNu%0v@KRMkYwxC;-OzeHM##%>`ji$rdm5mlgv4{mxA>bPr6gW2 z2t7dd8Hslugn9hrGwRD_UvHhsngk)bB?zr2&_{k(pxYfnXZwH@wxtE5Con>OhkIm= z{sH@bf|zZcpOB_){Z#=)zW%D`$rqH~9pHXfes937Gkb62IzqWsCkTL@3Jhg$6PO-S zOo7c|$%h#uSt-9$(Xpn0xHk2o+AVo&;X@^b&+g#WSB}Q>V;Ro#_ z$0Zd(7*j~0!z4dPR$GFyJ?A+9O52LtMSL6aYJly(}h;$`<`Y|ap#kUBA@>rdlvOS)~%ElxSO7`K6|J@Dypv6!BM8DE-YgGsNb;U z$V!1$N=(;oRVZA6F=Z%qqYg_BG68}iD=M-#w=fGV^^qDsS=%FeWs~W9X+Fxdy0BVh z(T95(@A+>}%-sJ=t;W(`J2*nNWw^564yx8iran!epNx3dI(+^K3kr+?fb9h^ zx&wH<$>s8EUY|@})mav2z z0gNkwV5yEu&W1>jtf*-w{%U@LC<*ifwleU1|M83vKyzn=9C2C7kA?Rnu#;&>CmAjR zm?%h|-4+Sl>Eb2z6F3P5m>l8v0reQ{89#3VUxL2kmpns#3m5TK5f~DP0z-Wp5CU*w zr-7ap6R#RoPTxef1gKGVm%LmIn*={1$k+A%z4JdjGsxuskh4>TP@ZNHYjxnL;bj&9 zdJCvzqP$!vhm2popc)wXC#1{O3v^rGns&u3Qw2gjIW&^`Zbq>n&jJo%riHWfDT6&X zcI*K&k9j=zG>*h8qUWin1{@X~x|3g^-4j;jHy%;2UkBVKt8R9A$NJ*lR(EUXqcy^Q zce~x9$y}aJ9b%n-lw=(bxvZ@4isx835a;aIIS92~T%4H5^j=@c-5MzTwh>ydDfrfh zhiWZ36Lo_+{lQMdT@`)j=tU7B*8XC)&?DMJ`@0*>a>CE1q{!NyAKMs_3*QE>4AkY) z2K>Ou>Qw?cL|EHgkkK2I>Eti=rO#q&&|H zf+gjuW_UAkUp_xxd;B=73tc1^F1RETRgpJwv(>9z&@dr4E!8wv$q~B{aqz0p*Rr>_ zHUpuXe)yTeAGJ&To*SQOFAsO+Xy58 literal 0 HcmV?d00001 diff --git a/docs/img/prefect-logo-white.png b/docs/img/prefect-logo-white.png deleted file mode 100644 index 50ca61396fa83406b0467ec53ed9a20d1e05a652..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2214 zcmV;X2wC@uP)A)7on2KI6I}Ub7KWsUMYqL|4P*$Xxetf}(u;>0|AzXf(TuXXgm-OW!-1JzX zR-R;Il1mK`ZW@*(*{ZYz>jT22!4$Nv=?| z31|O>zQTB|ms9U2X@d2kNjNLn8OCd^%qE$S7U3XUGiy38XLCyCqd_<~1_57il0D;g zPNd)J0kVCe^41(#q*()^*Wn=BJvA-=*z@s_!vS?6RXE6YFWNT8QbtN73Rk*p{Ggj>nBt=En~M+Z{hVJ36i3lxb%yAf(a2$%cB`LEnT#>h5NV27WksKtA-FN zoOJs3eyuD&kLZ4ZYy;^Nu2%ANmXkm!p3>MZv_0yi)UJ=Q^a-aS-Btsye0oexk$vs$ zn-crkV$m*KOq1w~j^&dD{XQ5-mvBo3lZ8!W`}UKT7Wh{Qj;%<#gp;0bb)u`j?AY%o zZ}~*UEf?mZs~=AK&O++zJKgdLW~33cCEQB*&cb4%0X<}kq)Rx-lyXh0I5Y~k)OueI z4KxY25MEBvM#Y#W;gD`%GPDVYbYsq;aFyWOEFj$ka44MQje1;fApkAH)!0avffnH~ zb`ydxoYe7~rVD)3;|nKsvarIIlN@~Eq?S`W*Ip3Pg~Di&$N)ws=qY*IWCIwvTn!jU#kT4S?bK9Yn( zx+y?XN!9)w{34JvX@GDFkW^A}8wUkQ5f1651PBMRB1Je2&-yAJgMQET*q-B4_74F`Bb}4 zO&!0VnEmBq+iiNRXs_wU-N{qUr%5<&Wfj6rhcDWBjt@L6sfJ;~VObtR@mqvu5zWyi9H(*Ip5Z?LDfps&)44GRO=CDi z1NBccF9PrBB-#J2jWJJWJQis*_M&m1O*k{7a69_(Qx?5?VO&tjWG9-VPq>`y=tMOw zgS7Tnf&UoPA~Yut?(XioKK_)-*_3pN=F@qgg(5~YM_0lbO1Y_o3l+_Ag+0~(5ht1> zM7ZmO_h`E%(LRjWLD&^8Ty!rQ2l|9F^cTI2Jl0#1>_f!Pflb4gk;fsC$vQXYU~1t~ zh~^(hNvIOd(Kl&u44pir6wRlEiGxbf93cs3C|yE?yC%tY6K%TN18@#vHFep$R4CkRof~ryT2dW$EVuDGf^&)HfBs*YK5J@<63sDGI1X6JA4xlC zk0M2LNW5)6+kL_wX75GgfJEWmF*t@Hu<1qPK z^vDURk3Qm9n*M-j{Yadp8X9>2&EPnURO63Gd-r-AyESkZnbxshlIkp6()sTP5K(ca+xCh3q|QXMJxR#o?+aX`Z( z$Wu%nu8?SX(2#I3lZT=^H|9V?!i||cWWO&OxJQFYq#oG`}x3oR;Ny^${$Q~&?~07*qoM6N<$f?w7gpa1{> diff --git a/docs/index.md b/docs/index.md index fdb966ca..091b54e9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -1,26 +1,24 @@ -# Coordinate and incorporate AWS in your dataflow with `prefect-aws` +# `prefect-aws`

- -
- PyPI + PyPI - + - + - +
- + - +

## Welcome! -The `prefect-aws` collection makes it easy to leverage the capabilities of AWS in your flows, featuring support for ECSTask, S3, Secrets Manager, Batch Job, and Client Waiter. +`prefect-aws` makes it easy to leverage the capabilities of AWS in your flows, featuring support for ECSTask, S3, Secrets Manager, Batch Job, and Client Waiter. ## Getting Started diff --git a/docs/stylesheets/extra.css b/docs/stylesheets/extra.css index 11a02095..662cca0c 100644 --- a/docs/stylesheets/extra.css +++ b/docs/stylesheets/extra.css @@ -1,9 +1,9 @@ /* theme */ :root > * { /* theme */ - --md-primary-fg-color: #115AF4; - --md-primary-fg-color--light: #115AF4; - --md-primary-fg-color--dark: #115AF4; + --md-primary-fg-color: #26272B; + --md-primary-fg-color--light: #26272B; + --md-primary-fg-color--dark: #26272B; } /* Table formatting */ @@ -72,7 +72,7 @@ to force column width */ /* dark mode slate theme */ /* dark mode code overrides */ [data-md-color-scheme="slate"] { - --md-code-bg-color: #252a33; + --md-code-bg-color: #1c1d20; --md-code-fg-color: #eee; --md-code-hl-color: #3b3d54; --md-code-hl-name-color: #eee; @@ -100,15 +100,15 @@ to force column width */ /* dark mode collection catalog overrides */ [data-md-color-scheme="slate"] .collection-item { - background-color: #3b3d54; + background-color: #26272B; } /* dark mode recipe collection overrides */ [data-md-color-scheme="slate"] .recipe-item { - background-color: #3b3d54; + background-color: #26272B; } /* dark mode API doc overrides */ [data-md-color-scheme="slate"] .prefect-table th { - background-color: #3b3d54; + background-color: #26272B; } \ No newline at end of file diff --git a/mkdocs.yml b/mkdocs.yml index 246fc31d..465f6407 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -28,7 +28,7 @@ theme: icon: repo: fontawesome/brands/github logo: - img/prefect-logo-mark-solid-white-500.png + img/prefect-logo-mark.png font: text: Inter code: Source Code Pro diff --git a/prefect_aws/credentials.py b/prefect_aws/credentials.py index 7810f7f9..64f49efe 100644 --- a/prefect_aws/credentials.py +++ b/prefect_aws/credentials.py @@ -40,7 +40,7 @@ class AwsCredentials(CredentialsBlock): ``` """ # noqa E501 - _logo_url = "https://images.ctfassets.net/gm98wzqotmnx/1jbV4lceHOjGgunX15lUwT/db88e184d727f721575aeb054a37e277/aws.png?h=250" # noqa + _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/d74b16fe84ce626345adf235a47008fea2869a60-225x225.png" # noqa _block_type_name = "AWS Credentials" _documentation_url = "https://prefecthq.github.io/prefect-aws/credentials/#prefect_aws.credentials.AwsCredentials" # noqa @@ -164,7 +164,7 @@ class MinIOCredentials(CredentialsBlock): ``` """ # noqa E501 - _logo_url = "https://images.ctfassets.net/gm98wzqotmnx/22vXcxsOrVeFrUwHfSoaeT/7607b876eb589a9028c8126e78f4c7b4/imageedit_7_2837870043.png?h=250" # noqa + _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/676cb17bcbdff601f97e0a02ff8bcb480e91ff40-250x250.png" # noqa _block_type_name = "MinIO Credentials" _description = ( "Block used to manage authentication with MinIO. Refer to the MinIO " diff --git a/prefect_aws/ecs.py b/prefect_aws/ecs.py index 7dbfa68a..a6ebe206 100644 --- a/prefect_aws/ecs.py +++ b/prefect_aws/ecs.py @@ -276,7 +276,7 @@ class ECSTask(Infrastructure): _block_type_slug = "ecs-task" _block_type_name = "ECS Task" - _logo_url = "https://images.ctfassets.net/gm98wzqotmnx/1jbV4lceHOjGgunX15lUwT/db88e184d727f721575aeb054a37e277/aws.png?h=250" # noqa + _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/d74b16fe84ce626345adf235a47008fea2869a60-225x225.png" # noqa _description = "Run a command as an ECS task." # noqa _documentation_url = ( "https://prefecthq.github.io/prefect-aws/ecs/#prefect_aws.ecs.ECSTask" # noqa diff --git a/prefect_aws/s3.py b/prefect_aws/s3.py index df84873c..11ca0438 100644 --- a/prefect_aws/s3.py +++ b/prefect_aws/s3.py @@ -404,7 +404,7 @@ class S3Bucket(WritableFileSystem, WritableDeploymentStorage, ObjectStorageBlock for reading and writing objects. """ - _logo_url = "https://images.ctfassets.net/gm98wzqotmnx/1jbV4lceHOjGgunX15lUwT/db88e184d727f721575aeb054a37e277/aws.png?h=250" # noqa + _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/d74b16fe84ce626345adf235a47008fea2869a60-225x225.png" # noqa _block_type_name = "S3 Bucket" _documentation_url = ( "https://prefecthq.github.io/prefect-aws/s3/#prefect_aws.s3.S3Bucket" # noqa diff --git a/prefect_aws/secrets_manager.py b/prefect_aws/secrets_manager.py index 5eea5ba8..9b93e867 100644 --- a/prefect_aws/secrets_manager.py +++ b/prefect_aws/secrets_manager.py @@ -368,7 +368,7 @@ class AwsSecret(SecretBlock): secret_name: The name of the secret. """ - _logo_url = "https://images.ctfassets.net/gm98wzqotmnx/1jbV4lceHOjGgunX15lUwT/db88e184d727f721575aeb054a37e277/aws.png?h=250" # noqa + _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/d74b16fe84ce626345adf235a47008fea2869a60-225x225.png" # noqa _block_type_name = "AWS Secret" _documentation_url = "https://prefecthq.github.io/prefect-aws/secrets_manager/#prefect_aws.secrets_manager.AwsSecret" # noqa diff --git a/prefect_aws/workers/ecs_worker.py b/prefect_aws/workers/ecs_worker.py index a3822a3f..02c4117c 100644 --- a/prefect_aws/workers/ecs_worker.py +++ b/prefect_aws/workers/ecs_worker.py @@ -548,7 +548,7 @@ class ECSWorker(BaseWorker): ) _display_name = "AWS Elastic Container Service" _documentation_url = "https://prefecthq.github.io/prefect-aws/ecs_worker/" - _logo_url = "https://images.ctfassets.net/gm98wzqotmnx/1jbV4lceHOjGgunX15lUwT/db88e184d727f721575aeb054a37e277/aws.png?h=250" # noqa + _logo_url = "https://cdn.sanity.io/images/3ugk85nk/production/d74b16fe84ce626345adf235a47008fea2869a60-225x225.png" # noqa async def run( self, diff --git a/requirements-dev.txt b/requirements-dev.txt index e22e9290..bcbdc906 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -17,6 +17,7 @@ pillow pre-commit pytest pytest-asyncio +pytest-cov pytest-lazy-fixture pytest-xdist types-boto3 >= 1.0.2 diff --git a/tests/test_ecs.py b/tests/test_ecs.py index a27ff925..cf18bfe4 100644 --- a/tests/test_ecs.py +++ b/tests/test_ecs.py @@ -1368,7 +1368,7 @@ async def test_latest_task_definition_not_used_if_inequal( # {"execution_role_arn": "test"}, # {"launch_type": "EXTERNAL"}, ], - ids=lambda item: str(set(item.keys())), + ids=lambda item: str(sorted(list(set(item.keys())))), ) async def test_latest_task_definition_with_overrides_that_do_not_require_copy( aws_credentials, overrides, launch_type @@ -1536,7 +1536,7 @@ async def test_task_definition_arn_with_overrides_requiring_copy_shows_diff( # from the base task definition {"env": {"FOO": None}}, ], - ids=lambda item: str(set(item.keys())), + ids=lambda item: str(sorted(list(set(item.keys())))), ) async def test_task_definition_arn_with_overrides_that_do_not_require_copy( aws_credentials, overrides diff --git a/tests/workers/test_ecs_worker.py b/tests/workers/test_ecs_worker.py index 2c177aae..b6a39b35 100644 --- a/tests/workers/test_ecs_worker.py +++ b/tests/workers/test_ecs_worker.py @@ -1587,7 +1587,7 @@ async def test_worker_task_definition_cache_miss_on_deregistered( # {"execution_role_arn": "test"}, # {"launch_type": "EXTERNAL"}, ], - ids=lambda item: str(set(item.keys())), + ids=lambda item: str(sorted(list(set(item.keys())))), ) async def test_worker_task_definition_cache_hit_on_config_changes( aws_credentials: AwsCredentials, From b6eb321bd65ffb7b1cff1a357540a58de7a66d88 Mon Sep 17 00:00:00 2001 From: urimandujano Date: Tue, 31 Oct 2023 09:34:03 -0500 Subject: [PATCH 06/11] 'Update the pin on `prefect` version' (#331) --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 40b6007d..919ce567 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,5 +2,5 @@ boto3>=1.24.53 botocore>=1.27.53 mypy_boto3_s3>=1.24.94 mypy_boto3_secretsmanager>=1.26.49 -prefect>=2.10.11 +prefect>=2.13.5 tenacity>=8.0.0 \ No newline at end of file From 3a5124fc383d94da6342af2b86588a1766c63591 Mon Sep 17 00:00:00 2001 From: urimandujano Date: Thu, 2 Nov 2023 11:56:12 -0500 Subject: [PATCH 07/11] Bumps build's python version (#334) --- .github/workflows/release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index cb1465e9..4b31753b 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -15,7 +15,7 @@ jobs: - name: Set up Python uses: actions/setup-python@v4 with: - python-version: 3.7 + python-version: 3.8 - name: Install packages run: | From f8a26eb0edc0603077da5ddd365b1e131ff6888b Mon Sep 17 00:00:00 2001 From: Justin T Date: Mon, 6 Nov 2023 20:22:20 +0100 Subject: [PATCH 08/11] fix ssl default (#328) Co-authored-by: Alexander Streed --- CHANGELOG.md | 8 ++++++++ prefect_aws/deployments/steps.py | 2 +- tests/deploments/test_steps.py | 4 ++-- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ec431ba7..d6d86da1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,6 +17,14 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ### Removed +## 0.4.2 + +Released November 6th, 2023. + +### Fixed + +- Fixed use_ssl default for s3 client. + ## 0.4.1 Released October 13th, 2023. diff --git a/prefect_aws/deployments/steps.py b/prefect_aws/deployments/steps.py index 77930465..7525a5e2 100644 --- a/prefect_aws/deployments/steps.py +++ b/prefect_aws/deployments/steps.py @@ -220,7 +220,7 @@ def get_s3_client( aws_client_parameters = credentials.get("aws_client_parameters", client_parameters) api_version = aws_client_parameters.get("api_version", None) endpoint_url = aws_client_parameters.get("endpoint_url", None) - use_ssl = aws_client_parameters.get("use_ssl", None) + use_ssl = aws_client_parameters.get("use_ssl", True) verify = aws_client_parameters.get("verify", None) config_params = aws_client_parameters.get("config", {}) config = Config(**config_params) diff --git a/tests/deploments/test_steps.py b/tests/deploments/test_steps.py index c78c3578..22608bd7 100644 --- a/tests/deploments/test_steps.py +++ b/tests/deploments/test_steps.py @@ -228,7 +228,7 @@ def test_s3_session_with_params(): assert { "api_version": "v1", "endpoint_url": None, - "use_ssl": None, + "use_ssl": True, "verify": None, }.items() <= all_calls[1].kwargs.items() assert all_calls[1].kwargs.get("config").connect_timeout == 300 @@ -244,7 +244,7 @@ def test_s3_session_with_params(): assert { "api_version": None, "endpoint_url": None, - "use_ssl": None, + "use_ssl": True, "verify": None, }.items() <= all_calls[3].kwargs.items() assert all_calls[3].kwargs.get("config").connect_timeout == 60 From 679fad1c6a55a30240212302d0f579d42dbab445 Mon Sep 17 00:00:00 2001 From: nick-amplify <131465708+nick-amplify@users.noreply.github.com> Date: Mon, 13 Nov 2023 10:37:02 -0500 Subject: [PATCH 09/11] Read AWS Secrets with 'SecretString' (#274) Co-authored-by: ntorba <32570754+ntorba@users.noreply.github.com> Co-authored-by: Alexander Streed Co-authored-by: Alexander Streed --- CHANGELOG.md | 1 + prefect_aws/secrets_manager.py | 5 ++++- tests/test_secrets_manager.py | 7 +++++++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d6d86da1..d4e01351 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ### Added ### Changed +- Added 'SecretBrinary' suport to `AwsSecret` block - [#274](https://github.com/PrefectHQ/prefect-aws/pull/274) ### Fixed diff --git a/prefect_aws/secrets_manager.py b/prefect_aws/secrets_manager.py index 9b93e867..c043f28c 100644 --- a/prefect_aws/secrets_manager.py +++ b/prefect_aws/secrets_manager.py @@ -411,7 +411,10 @@ async def read_secret( response = await run_sync_in_worker_thread( client.get_secret_value, SecretId=self.secret_name, **read_kwargs ) - secret = response["SecretBinary"] + if "SecretBinary" in response: + secret = response["SecretBinary"] + elif "SecretString" in response: + secret = response["SecretString"] arn = response["ARN"] self.logger.info(f"The secret {arn!r} data was successfully read.") return secret diff --git a/tests/test_secrets_manager.py b/tests/test_secrets_manager.py index 03d0a08b..654f0576 100644 --- a/tests/test_secrets_manager.py +++ b/tests/test_secrets_manager.py @@ -199,3 +199,10 @@ def test_delete_secret_recovery_window(self, aws_secret: AwsSecret): ValueError, match="Recovery window must be between 7 and 30 days" ): aws_secret.delete_secret(recovery_window_in_days=42) + + async def test_read_secret(self, secret_under_test, aws_credentials): + secret = AwsSecret( + aws_credentials=aws_credentials, + secret_name=secret_under_test["secret_name"], + ) + assert await secret.read_secret() == secret_under_test["expected_value"] From 30e358fb9e0f617e13fd4eda5ccbc7b5572cb6a8 Mon Sep 17 00:00:00 2001 From: kevingrismore <146098880+kevingrismore@users.noreply.github.com> Date: Wed, 15 Nov 2023 11:55:43 -0500 Subject: [PATCH 10/11] ecs guide fixes and clarifications (#329) Co-authored-by: nate nowack --- docs/ecs_guide.md | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/docs/ecs_guide.md b/docs/ecs_guide.md index 33bcfccb..5ac31ae2 100644 --- a/docs/ecs_guide.md +++ b/docs/ecs_guide.md @@ -182,22 +182,14 @@ To create an [IAM role](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_role "containerDefinitions": [ { "name": "prefect-worker", - "image": "prefecthq/prefect", + "image": "prefecthq/prefect:2-latest", "cpu": 512, "memory": 1024, "essential": true, "command": [ - "pip", - "install", - "prefect-aws", - "&&", - "prefect", - "worker", - "start", - "--pool", - "my-ecs-pool", - "--type", - "ecs" + "/bin/sh", + "-c", + "pip install prefect-aws && prefect worker start --pool my-ecs-pool --type ecs" ], "environment": [ { @@ -218,7 +210,7 @@ To create an [IAM role](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_role - For the `PREFECT_API_KEY`, individuals on the organization tier can create a [service account](https://docs.prefect.io/latest/cloud/users/service-accounts/) for the worker. If on a personal tier, you can pass a user’s API key. - - Replace `` with the ARN of the IAM role you created in Step 1. + - Replace both instances of `` with the ARN of the IAM role you created in Step 2. - Notice that the CPU and Memory allocations are relatively small. The worker's main responsibility is to submit work through API calls to AWS, _not_ to execute your Prefect flow code. From 36b203dff4b04b988251a6c1bbdbba1c28b210c6 Mon Sep 17 00:00:00 2001 From: Jeff Hale Date: Thu, 16 Nov 2023 15:08:06 -0500 Subject: [PATCH 11/11] Update ecs guide to remove pinned Prefect docs references (#338) --- docs/ecs_guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ecs_guide.md b/docs/ecs_guide.md index 5ac31ae2..d188a89a 100644 --- a/docs/ecs_guide.md +++ b/docs/ecs_guide.md @@ -11,7 +11,7 @@ ECS (Elastic Container Service) tasks are a good option for executing Prefect 2 ## ECS Flow Run Execution -Prefect enables remote flow execution via [workers](https://docs.prefect.io/2.11.1/concepts/work-pools/#worker-overview) and [work pools](https://docs.prefect.io/2.11.1/concepts/work-pools/#work-pool-overview). To learn more about these concepts please see our [deployment tutorial](https://docs.prefect.io/2.11.1/tutorial/deployments/). +Prefect enables remote flow execution via [workers](https://docs.prefect.io/concepts/work-pools/#worker-overview) and [work pools](https://docs.prefect.io/concepts/work-pools/#work-pool-overview). To learn more about these concepts please see our [deployment tutorial](https://docs.prefect.io/tutorial/deployments/). For details on how workers and work pools are implemented for ECS, see the diagram below: #### Architecture Diagram @@ -288,10 +288,10 @@ To create an [IAM role](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_role - Do your flow runs require higher `CPU`? - Would an EC2 `Launch Type` speed up your flow run execution? - These infrastructure configuration values can be set on your ECS work pool or they can be overridden on the deployment level through [job_variables](https://docs.prefect.io/2.11.0/concepts/infrastructure/#kubernetesjob-overrides-and-customizations) if desired. + These infrastructure configuration values can be set on your ECS work pool or they can be overridden on the deployment level through [job_variables](https://docs.prefect.io/concepts/infrastructure/#kubernetesjob-overrides-and-customizations) if desired. -2. Consider adding a [build action](https://docs.prefect.io/2.11.0/concepts/deployments-ux/#the-build-action) to your Prefect Project [`prefect.yaml`](https://docs.prefect.io/2.11.0/concepts/deployments-ux/#the-prefect-yaml-file) if you want to automatically build a Docker image and push it to an image registry `prefect deploy` is run. +2. Consider adding a [build action](https://docs.prefect.io/concepts/deployments-ux/#the-build-action) to your Prefect Project [`prefect.yaml`](https://docs.prefect.io/concepts/deployments-ux/#the-prefect-yaml-file) if you want to automatically build a Docker image and push it to an image registry `prefect deploy` is run. Here is an example build action for ECR: ```yaml