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`
-
-
-
+
-
+
-
+
-
+
-
+
-
+
## 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@0zLRvLgtHu7KVvdv|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`
-
-
-
+
-
+
-
+
-
+
-
+
-
+
## 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