From c66e2a5f270a5848e9e261c061a7f17ff39cb08c Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 25 Nov 2024 15:41:43 -0500 Subject: [PATCH 01/24] initial commit, try to fix pickling error --- .../sources/community/blob_store/__init__.py | 4 ++ .../community/blob_store/blob_source.py | 57 ++++++++++++++++++ .../community/blob_store/clients/__init__.py | 3 + .../community/blob_store/clients/aws.py | 58 +++++++++++++++++++ .../community/blob_store/clients/azure.py | 34 +++++++++++ .../community/blob_store/clients/base.py | 30 ++++++++++ .../blob_store/compressions/__init__.py | 2 + .../community/blob_store/formats/__init__.py | 2 + .../community/file/compressions/base.py | 17 +++++- .../community/file/compressions/gzip.py | 10 ++-- .../sources/community/file/formats/base.py | 37 ++++++------ 11 files changed, 228 insertions(+), 26 deletions(-) create mode 100644 quixstreams/sources/community/blob_store/__init__.py create mode 100644 quixstreams/sources/community/blob_store/blob_source.py create mode 100644 quixstreams/sources/community/blob_store/clients/__init__.py create mode 100644 quixstreams/sources/community/blob_store/clients/aws.py create mode 100644 quixstreams/sources/community/blob_store/clients/azure.py create mode 100644 quixstreams/sources/community/blob_store/clients/base.py create mode 100644 quixstreams/sources/community/blob_store/compressions/__init__.py create mode 100644 quixstreams/sources/community/blob_store/formats/__init__.py diff --git a/quixstreams/sources/community/blob_store/__init__.py b/quixstreams/sources/community/blob_store/__init__.py new file mode 100644 index 000000000..e61e17c6c --- /dev/null +++ b/quixstreams/sources/community/blob_store/__init__.py @@ -0,0 +1,4 @@ +# ruff: noqa: F403 +from .blob_source import * +from .clients import * +from .formats import * diff --git a/quixstreams/sources/community/blob_store/blob_source.py b/quixstreams/sources/community/blob_store/blob_source.py new file mode 100644 index 000000000..e05655ff4 --- /dev/null +++ b/quixstreams/sources/community/blob_store/blob_source.py @@ -0,0 +1,57 @@ +import logging +from pathlib import Path +from typing import Optional, Union + +from quixstreams.sources.community.file import FileSource + +from .clients.base import BlobClient +from .compressions import CompressionName +from .formats import Format, FormatName + +logger = logging.getLogger(__name__) + + +__all__ = ("BlobFileSource",) + + +class BlobFileSource(FileSource): + def __init__( + self, + blob_client: BlobClient, + blob_format: Union[FormatName, Format], + blob_compression: Optional[CompressionName] = None, + blob_folder: Optional[str] = None, + blob_file: Optional[str] = None, + as_replay: bool = True, + name: Optional[str] = None, + shutdown_timeout: float = 10.0, + ): + self._blob_client = blob_client + if blob_file: + self._blobs = lambda: [blob_file] + else: + self._blobs = lambda: self._blob_client.blob_finder(blob_folder) + + super().__init__( + filepath=self._blob_client.location, + file_format=blob_format, + file_compression=blob_compression, + as_replay=as_replay, + name=name or self._blob_client.location, + shutdown_timeout=shutdown_timeout, + ) + + def _get_partition_count(self) -> int: + return 1 + + def run(self): + while self._running: + for file in self._blobs(): + self._check_file_partition_number(Path(file)) + filestream = self._blob_client.get_raw_blob(file) + for record in self._formatter.file_read(filestream): + if self._as_replay and (timestamp := record.get("_timestamp")): + self._replay_delay(timestamp) + self._produce(record) + self.flush() + return diff --git a/quixstreams/sources/community/blob_store/clients/__init__.py b/quixstreams/sources/community/blob_store/clients/__init__.py new file mode 100644 index 000000000..1bac3f560 --- /dev/null +++ b/quixstreams/sources/community/blob_store/clients/__init__.py @@ -0,0 +1,3 @@ +# ruff: noqa: F403 +from .aws import * +from .azure import * diff --git a/quixstreams/sources/community/blob_store/clients/aws.py b/quixstreams/sources/community/blob_store/clients/aws.py new file mode 100644 index 000000000..d90878ebe --- /dev/null +++ b/quixstreams/sources/community/blob_store/clients/aws.py @@ -0,0 +1,58 @@ +from typing import Generator, Optional + +from .base import BlobClient + +try: + from boto3 import client as boto_client + from mypy_boto3_s3 import S3Client +except Exception: + raise + + +__all__ = ("AwsS3BlobClient",) + + +class AwsS3BlobClient(BlobClient): + def __init__( + self, + aws_s3_bucket: str, + region_name: Optional[str] = None, + aws_access_key_id: Optional[str] = None, + aws_secret_access_key: Optional[str] = None, + aws_session_token: Optional[str] = None, + ): + """ + Configure IcebergSink to work with AWS Glue. + + :param aws_s3_bucket: The S3 URI where the table data will be stored + (e.g., 's3://your-bucket/warehouse/'). + :param region_name: The AWS region for the S3 bucket + NOTE: can alternatively set AWS_REGION environment variable + :param aws_access_key_id: the AWS access key ID. + NOTE: can alternatively set the AWS_ACCESS_KEY_ID environment variable + :param aws_secret_access_key: the AWS secret access key. + NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable + :param aws_session_token: a session token (or will be generated for you). + NOTE: can alternatively set the AWS_SESSION_TOKEN environment variable. + """ + self.location = aws_s3_bucket + self.client: S3Client = boto_client( + "s3", + region_name=region_name, + aws_access_key_id=aws_access_key_id, + aws_secret_access_key=aws_secret_access_key, + aws_session_token=aws_session_token, + ) + + def get_raw_blob(self, blob_path: str) -> bytes: + return self.client.get_object(Bucket=self.location, Key=blob_path)[ + "Body" + ].read() + + def blob_finder(self, folder: str) -> Generator[str, None, None]: + # TODO: Recursively navigate through folders. + resp = self.client.list_objects( + Bucket=self.location, Prefix=folder, Delimiter="/" + ) + for item in resp["Contents"]: + yield item["Key"] diff --git a/quixstreams/sources/community/blob_store/clients/azure.py b/quixstreams/sources/community/blob_store/clients/azure.py new file mode 100644 index 000000000..8c2960721 --- /dev/null +++ b/quixstreams/sources/community/blob_store/clients/azure.py @@ -0,0 +1,34 @@ +from io import BytesIO +from typing import Generator + +from .base import BlobClient + +try: + from azure.storage.blob import BlobServiceClient + from azure.storage.blob._container_client import ContainerClient +except Exception: + raise + +__all__ = ("AzureBlobClient",) + + +class AzureBlobClient(BlobClient): + def __init__( + self, + connection_string: str, + container: str, + ): + blob_client = BlobServiceClient.from_connection_string(connection_string) + container_client = blob_client.get_container_client(container) + + self.client: ContainerClient = container_client + self.location = container + + def blob_finder(self, folder: str) -> Generator[str, None, None]: + for item in self.client.list_blob_names(name_starts_with=folder): + yield item + + def get_raw_blob(self, blob_name) -> BytesIO: + blob_client = self.client.get_blob_client(blob_name) + data: bytes = blob_client.download_blob().readall() + return BytesIO(data) diff --git a/quixstreams/sources/community/blob_store/clients/base.py b/quixstreams/sources/community/blob_store/clients/base.py new file mode 100644 index 000000000..dd39aefe3 --- /dev/null +++ b/quixstreams/sources/community/blob_store/clients/base.py @@ -0,0 +1,30 @@ +from abc import abstractmethod +from dataclasses import dataclass +from io import BytesIO +from typing import Any, Iterable + +__all__ = ("BlobClient",) + + +@dataclass +class BlobClient: + client: Any + location: str + + @abstractmethod + def blob_finder(self, folder: str) -> Iterable[str]: ... + + """ + Find all blobs starting from a root folder. + + Each item in the iterable should be a filepath resolvable by `get_raw_blob`. + """ + + @abstractmethod + def get_raw_blob(self, blob_path: str) -> BytesIO: ... + + """ + Obtain a specific blob in its raw form. + + Result should be ready for deserialization (and/or decompression). + """ diff --git a/quixstreams/sources/community/blob_store/compressions/__init__.py b/quixstreams/sources/community/blob_store/compressions/__init__.py new file mode 100644 index 000000000..0cf3be105 --- /dev/null +++ b/quixstreams/sources/community/blob_store/compressions/__init__.py @@ -0,0 +1,2 @@ +# ruff: noqa: F403 +from quixstreams.sources.community.file.compressions import * diff --git a/quixstreams/sources/community/blob_store/formats/__init__.py b/quixstreams/sources/community/blob_store/formats/__init__.py new file mode 100644 index 000000000..7df3748e7 --- /dev/null +++ b/quixstreams/sources/community/blob_store/formats/__init__.py @@ -0,0 +1,2 @@ +# ruff: noqa: F403 +from quixstreams.sources.community.file.formats import * diff --git a/quixstreams/sources/community/file/compressions/base.py b/quixstreams/sources/community/file/compressions/base.py index c4b90b7e6..0f302d958 100644 --- a/quixstreams/sources/community/file/compressions/base.py +++ b/quixstreams/sources/community/file/compressions/base.py @@ -1,6 +1,7 @@ +import contextlib from abc import ABC, abstractmethod from pathlib import Path -from typing import Literal +from typing import BinaryIO, Literal, Union __all__ = ( "Decompressor", @@ -13,4 +14,16 @@ class Decompressor(ABC): @abstractmethod - def decompress(self, filepath: Path) -> bytes: ... + def _decompress(self, filestream: BinaryIO) -> bytes: ... + + @contextlib.contextmanager + def _open(self, file: Union[Path, BinaryIO]) -> BinaryIO: + if isinstance(file, Path): + with open(file, "rb") as f: + yield f + else: + yield file + + def decompress(self, file: Union[Path, BinaryIO]) -> bytes: + with self._open(file) as filestream: + return self._decompress(filestream) diff --git a/quixstreams/sources/community/file/compressions/gzip.py b/quixstreams/sources/community/file/compressions/gzip.py index 5bfc9aeea..50cda2d82 100644 --- a/quixstreams/sources/community/file/compressions/gzip.py +++ b/quixstreams/sources/community/file/compressions/gzip.py @@ -1,4 +1,5 @@ -from pathlib import Path +from gzip import decompress +from typing import BinaryIO from .base import Decompressor @@ -7,10 +8,7 @@ class GZipDecompressor(Decompressor): def __init__(self): - from gzip import decompress - self._decompressor = decompress - def decompress(self, filepath: Path) -> bytes: - with open(filepath, "rb") as f: - return self._decompressor(f.read()) + def _decompress(self, filestream: BinaryIO) -> bytes: + return self._decompressor(filestream.read()) diff --git a/quixstreams/sources/community/file/formats/base.py b/quixstreams/sources/community/file/formats/base.py index 9a164e5ef..102ff9ba4 100644 --- a/quixstreams/sources/community/file/formats/base.py +++ b/quixstreams/sources/community/file/formats/base.py @@ -1,7 +1,8 @@ +import contextlib from abc import ABC, abstractmethod from io import BytesIO from pathlib import Path -from typing import BinaryIO, Generator, Iterable, Literal, Optional +from typing import BinaryIO, Generator, Iterable, Literal, Optional, Union from ..compressions import COMPRESSION_MAPPER, CompressionName, Decompressor @@ -26,7 +27,6 @@ def __init__(self, compression: Optional[CompressionName] = None): """ super().__init__() this for a usable init. """ - self._file: Optional[BinaryIO] = None self._decompressor: Optional[Decompressor] = None if compression: self._set_decompressor(compression) @@ -48,22 +48,23 @@ def deserialize(self, filestream: BinaryIO) -> Iterable[dict]: """ ... - def _set_decompressor(self, extension_or_name: CompressionName): - self._decompressor = COMPRESSION_MAPPER[extension_or_name]() - - def _open_filestream(self, filepath: Path): - # TODO: maybe check that file extension is valid? - if self._decompressor: - self._file = BytesIO(self._decompressor.decompress(filepath)) + def _decompress(self, filestream: BinaryIO) -> BinaryIO: + if not self._decompressor: + return filestream + return BytesIO(self._decompressor.decompress(filestream)) + + @contextlib.contextmanager + def _open(self, file: Union[Path, BinaryIO]) -> BinaryIO: + if isinstance(file, Path): + with open(file, "rb") as f: + # yield for when no decompression is done + yield self._decompress(f) else: - self._file = open(filepath, "rb") + yield self._decompress(file) - def _close_filestream(self): - if self._file: - self._file.close() - self._file = None + def _set_decompressor(self, extension_or_name: CompressionName): + self._decompressor = COMPRESSION_MAPPER[extension_or_name]() - def file_read(self, filepath: Path) -> Generator[dict, None, None]: - self._open_filestream(filepath) - yield from self.deserialize(self._file) - self._close_filestream() + def file_read(self, file: Union[Path, BinaryIO]) -> Generator[dict, None, None]: + with self._open(file) as filestream: + yield from self.deserialize(filestream) From 6389fcd8f3e0855e1ff2b0afd3bcf1f8f8afa0c8 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 25 Nov 2024 17:21:46 -0500 Subject: [PATCH 02/24] fixed issue around pickling, need to simplify file gathering and partition counting --- .../sources/community/blob_store/__init__.py | 3 +- .../community/blob_store/blob_source.py | 28 +++++++----- .../community/blob_store/clients/aws.py | 44 ++++++++++++------- .../community/blob_store/clients/azure.py | 20 ++++++--- .../community/blob_store/clients/base.py | 20 +++++++-- .../blob_store/compressions/__init__.py | 2 - .../community/blob_store/formats/__init__.py | 2 - 7 files changed, 76 insertions(+), 43 deletions(-) delete mode 100644 quixstreams/sources/community/blob_store/compressions/__init__.py delete mode 100644 quixstreams/sources/community/blob_store/formats/__init__.py diff --git a/quixstreams/sources/community/blob_store/__init__.py b/quixstreams/sources/community/blob_store/__init__.py index e61e17c6c..9466c031f 100644 --- a/quixstreams/sources/community/blob_store/__init__.py +++ b/quixstreams/sources/community/blob_store/__init__.py @@ -1,4 +1,5 @@ # ruff: noqa: F403 +from quixstreams.sources.community.file.formats import * + from .blob_source import * from .clients import * -from .formats import * diff --git a/quixstreams/sources/community/blob_store/blob_source.py b/quixstreams/sources/community/blob_store/blob_source.py index e05655ff4..3fa0199be 100644 --- a/quixstreams/sources/community/blob_store/blob_source.py +++ b/quixstreams/sources/community/blob_store/blob_source.py @@ -3,10 +3,10 @@ from typing import Optional, Union from quixstreams.sources.community.file import FileSource +from quixstreams.sources.community.file.compressions import CompressionName +from quixstreams.sources.community.file.formats import Format, FormatName from .clients.base import BlobClient -from .compressions import CompressionName -from .formats import Format, FormatName logger = logging.getLogger(__name__) @@ -26,29 +26,35 @@ def __init__( name: Optional[str] = None, shutdown_timeout: float = 10.0, ): - self._blob_client = blob_client - if blob_file: - self._blobs = lambda: [blob_file] - else: - self._blobs = lambda: self._blob_client.blob_finder(blob_folder) + self._client = blob_client + self._blob_file = blob_file + self._blob_folder = blob_folder super().__init__( - filepath=self._blob_client.location, + filepath=self._client.location, file_format=blob_format, file_compression=blob_compression, as_replay=as_replay, - name=name or self._blob_client.location, + name=name or self._client.location, shutdown_timeout=shutdown_timeout, ) def _get_partition_count(self) -> int: return 1 + def _check_file_partition_number(self, file: Path) -> int: + return 0 + def run(self): + blobs = ( + [self._blob_file] + if self._blob_file + else self._client.blob_finder(self._blob_folder) + ) while self._running: - for file in self._blobs(): + for file in blobs: self._check_file_partition_number(Path(file)) - filestream = self._blob_client.get_raw_blob(file) + filestream = self._client.get_raw_blob_stream(file) for record in self._formatter.file_read(filestream): if self._as_replay and (timestamp := record.get("_timestamp")): self._replay_delay(timestamp) diff --git a/quixstreams/sources/community/blob_store/clients/aws.py b/quixstreams/sources/community/blob_store/clients/aws.py index d90878ebe..ccc92e5e6 100644 --- a/quixstreams/sources/community/blob_store/clients/aws.py +++ b/quixstreams/sources/community/blob_store/clients/aws.py @@ -1,3 +1,5 @@ +from io import BytesIO +from os import getenv from typing import Generator, Optional from .base import BlobClient @@ -16,9 +18,10 @@ class AwsS3BlobClient(BlobClient): def __init__( self, aws_s3_bucket: str, - region_name: Optional[str] = None, - aws_access_key_id: Optional[str] = None, - aws_secret_access_key: Optional[str] = None, + aws_region: Optional[str] = getenv("AWS_REGION"), + aws_access_key_id: Optional[str] = getenv("AWS_ACCESS_KEY_ID"), + aws_secret_access_key: Optional[str] = getenv("AWS_SECRET_ACCESS_KEY"), + aws_endpoint_url: Optional[str] = getenv("AWS_ENDPOINT_URL_S3"), aws_session_token: Optional[str] = None, ): """ @@ -26,28 +29,37 @@ def __init__( :param aws_s3_bucket: The S3 URI where the table data will be stored (e.g., 's3://your-bucket/warehouse/'). - :param region_name: The AWS region for the S3 bucket - NOTE: can alternatively set AWS_REGION environment variable + :param aws_region: The AWS region. + NOTE: can alternatively set the AWS_REGION environment variable :param aws_access_key_id: the AWS access key ID. NOTE: can alternatively set the AWS_ACCESS_KEY_ID environment variable :param aws_secret_access_key: the AWS secret access key. NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable - :param aws_session_token: a session token (or will be generated for you). - NOTE: can alternatively set the AWS_SESSION_TOKEN environment variable. + :param aws_endpoint_url: the endpoint URL to use; only required for connecting + to a locally hosted Kinesis. + NOTE: can alternatively set the AWS_ENDPOINT_URL_KINESIS environment variable """ self.location = aws_s3_bucket - self.client: S3Client = boto_client( - "s3", - region_name=region_name, - aws_access_key_id=aws_access_key_id, - aws_secret_access_key=aws_secret_access_key, - aws_session_token=aws_session_token, - ) + self._client: Optional[S3Client] = None + self._credentials = { + "region_name": aws_region, + "aws_access_key_id": aws_access_key_id, + "aws_secret_access_key": aws_secret_access_key, + "aws_session_token": aws_session_token, + "endpoint_url": aws_endpoint_url, + } + + @property + def client(self): + if not self._client: + self._client: S3Client = boto_client("s3", **self._credentials) + return self._client - def get_raw_blob(self, blob_path: str) -> bytes: - return self.client.get_object(Bucket=self.location, Key=blob_path)[ + def get_raw_blob_stream(self, blob_path: str) -> BytesIO: + data = self.client.get_object(Bucket=self.location, Key=blob_path)[ "Body" ].read() + return BytesIO(data) def blob_finder(self, folder: str) -> Generator[str, None, None]: # TODO: Recursively navigate through folders. diff --git a/quixstreams/sources/community/blob_store/clients/azure.py b/quixstreams/sources/community/blob_store/clients/azure.py index 8c2960721..9590e9d2b 100644 --- a/quixstreams/sources/community/blob_store/clients/azure.py +++ b/quixstreams/sources/community/blob_store/clients/azure.py @@ -1,5 +1,5 @@ from io import BytesIO -from typing import Generator +from typing import Generator, Optional from .base import BlobClient @@ -18,17 +18,23 @@ def __init__( connection_string: str, container: str, ): - blob_client = BlobServiceClient.from_connection_string(connection_string) - container_client = blob_client.get_container_client(container) - - self.client: ContainerClient = container_client + self._client: Optional[ContainerClient] = None self.location = container + self._credentials = connection_string + + @property + def client(self): + if not self._client: + blob_client = BlobServiceClient.from_connection_string(self._credentials) + container_client = blob_client.get_container_client(self.location) + self._client: ContainerClient = container_client + return self._client def blob_finder(self, folder: str) -> Generator[str, None, None]: for item in self.client.list_blob_names(name_starts_with=folder): yield item - def get_raw_blob(self, blob_name) -> BytesIO: + def get_raw_blob_stream(self, blob_name) -> BytesIO: blob_client = self.client.get_blob_client(blob_name) - data: bytes = blob_client.download_blob().readall() + data = blob_client.download_blob().readall() return BytesIO(data) diff --git a/quixstreams/sources/community/blob_store/clients/base.py b/quixstreams/sources/community/blob_store/clients/base.py index dd39aefe3..5f3a90458 100644 --- a/quixstreams/sources/community/blob_store/clients/base.py +++ b/quixstreams/sources/community/blob_store/clients/base.py @@ -1,27 +1,39 @@ from abc import abstractmethod from dataclasses import dataclass from io import BytesIO -from typing import Any, Iterable +from typing import Any, Iterable, Union __all__ = ("BlobClient",) @dataclass class BlobClient: - client: Any + _client: Any + _credentials: Union[dict, str] location: str + @abstractmethod + @property + def client(self): ... + + """ + Set _client here. + + Circumvents pickling issues with multiprocessing by init'ing the client in a + later step (when Application.run() is called). + """ + @abstractmethod def blob_finder(self, folder: str) -> Iterable[str]: ... """ Find all blobs starting from a root folder. - Each item in the iterable should be a filepath resolvable by `get_raw_blob`. + Each item in the iterable should be a filepath resolvable by `get_raw_blob_stream`. """ @abstractmethod - def get_raw_blob(self, blob_path: str) -> BytesIO: ... + def get_raw_blob_stream(self, blob_path: str) -> BytesIO: ... """ Obtain a specific blob in its raw form. diff --git a/quixstreams/sources/community/blob_store/compressions/__init__.py b/quixstreams/sources/community/blob_store/compressions/__init__.py deleted file mode 100644 index 0cf3be105..000000000 --- a/quixstreams/sources/community/blob_store/compressions/__init__.py +++ /dev/null @@ -1,2 +0,0 @@ -# ruff: noqa: F403 -from quixstreams.sources.community.file.compressions import * diff --git a/quixstreams/sources/community/blob_store/formats/__init__.py b/quixstreams/sources/community/blob_store/formats/__init__.py deleted file mode 100644 index 7df3748e7..000000000 --- a/quixstreams/sources/community/blob_store/formats/__init__.py +++ /dev/null @@ -1,2 +0,0 @@ -# ruff: noqa: F403 -from quixstreams.sources.community.file.formats import * From fa4ad968d14837ff5b0c0c09cac476e9f5b8e818 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 25 Nov 2024 20:16:38 -0500 Subject: [PATCH 03/24] created more shared methods and unified classes more --- .../community/blob_store/blob_source.py | 37 ++++++----------- .../community/blob_store/clients/aws.py | 31 ++++++++++---- .../community/blob_store/clients/azure.py | 10 +++-- .../community/blob_store/clients/base.py | 14 +++++-- quixstreams/sources/community/file/file.py | 40 +++++++++++-------- 5 files changed, 75 insertions(+), 57 deletions(-) diff --git a/quixstreams/sources/community/blob_store/blob_source.py b/quixstreams/sources/community/blob_store/blob_source.py index 3fa0199be..914a9e40a 100644 --- a/quixstreams/sources/community/blob_store/blob_source.py +++ b/quixstreams/sources/community/blob_store/blob_source.py @@ -1,6 +1,6 @@ import logging from pathlib import Path -from typing import Optional, Union +from typing import Generator, Optional, Union from quixstreams.sources.community.file import FileSource from quixstreams.sources.community.file.compressions import CompressionName @@ -20,15 +20,15 @@ def __init__( blob_client: BlobClient, blob_format: Union[FormatName, Format], blob_compression: Optional[CompressionName] = None, - blob_folder: Optional[str] = None, - blob_file: Optional[str] = None, + blob_folder: Optional[Union[str, Path]] = None, + blob_file: Optional[Union[str, Path]] = None, as_replay: bool = True, name: Optional[str] = None, shutdown_timeout: float = 10.0, ): self._client = blob_client - self._blob_file = blob_file - self._blob_folder = blob_folder + self._blob_file = Path(blob_file) if blob_file else None + self._blob_folder = Path(blob_folder) if blob_folder else None super().__init__( filepath=self._client.location, @@ -40,24 +40,13 @@ def __init__( ) def _get_partition_count(self) -> int: - return 1 + return self._client.get_root_folder_count(self._blob_folder) - def _check_file_partition_number(self, file: Path) -> int: - return 0 + def _file_read(self, file: Path) -> Generator[dict, None, None]: + yield from super()._file_read(self._client.get_raw_blob_stream(file)) - def run(self): - blobs = ( - [self._blob_file] - if self._blob_file - else self._client.blob_finder(self._blob_folder) - ) - while self._running: - for file in blobs: - self._check_file_partition_number(Path(file)) - filestream = self._client.get_raw_blob_stream(file) - for record in self._formatter.file_read(filestream): - if self._as_replay and (timestamp := record.get("_timestamp")): - self._replay_delay(timestamp) - self._produce(record) - self.flush() - return + def _file_list(self) -> Generator[Path, None, None]: + if self._blob_file: + yield self._blob_file + else: + yield from self._client.blob_collector(self._blob_folder) diff --git a/quixstreams/sources/community/blob_store/clients/aws.py b/quixstreams/sources/community/blob_store/clients/aws.py index ccc92e5e6..6f747d8f9 100644 --- a/quixstreams/sources/community/blob_store/clients/aws.py +++ b/quixstreams/sources/community/blob_store/clients/aws.py @@ -1,6 +1,8 @@ +import logging from io import BytesIO from os import getenv -from typing import Generator, Optional +from pathlib import Path +from typing import Generator, Optional, Union from .base import BlobClient @@ -11,6 +13,8 @@ raise +logger = logging.getLogger(__name__) + __all__ = ("AwsS3BlobClient",) @@ -55,16 +59,27 @@ def client(self): self._client: S3Client = boto_client("s3", **self._credentials) return self._client - def get_raw_blob_stream(self, blob_path: str) -> BytesIO: - data = self.client.get_object(Bucket=self.location, Key=blob_path)[ + def get_raw_blob_stream(self, blob_path: Path) -> BytesIO: + data = self.client.get_object(Bucket=self.location, Key=str(blob_path))[ "Body" ].read() return BytesIO(data) - def blob_finder(self, folder: str) -> Generator[str, None, None]: - # TODO: Recursively navigate through folders. + def get_root_folder_count(self, folder: Path) -> int: resp = self.client.list_objects( - Bucket=self.location, Prefix=folder, Delimiter="/" + Bucket=self.location, Prefix=str(folder), Delimiter="/" ) - for item in resp["Contents"]: - yield item["Key"] + self._client = None + return len(resp["CommonPrefixes"]) + + def blob_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None]: + resp = self.client.list_objects( + Bucket=self.location, + Prefix=str(folder), + Delimiter="/", + ) + for folder in resp.get("CommonPrefixes", []): + yield from self.blob_collector(folder["Prefix"]) + + for file in resp.get("Contents", []): + yield Path(file["Key"]) diff --git a/quixstreams/sources/community/blob_store/clients/azure.py b/quixstreams/sources/community/blob_store/clients/azure.py index 9590e9d2b..2830135a6 100644 --- a/quixstreams/sources/community/blob_store/clients/azure.py +++ b/quixstreams/sources/community/blob_store/clients/azure.py @@ -1,4 +1,5 @@ from io import BytesIO +from pathlib import Path from typing import Generator, Optional from .base import BlobClient @@ -30,11 +31,12 @@ def client(self): self._client: ContainerClient = container_client return self._client - def blob_finder(self, folder: str) -> Generator[str, None, None]: - for item in self.client.list_blob_names(name_starts_with=folder): + def blob_collector(self, folder: Path) -> Generator[str, None, None]: + # TODO: Recursively navigate folders. + for item in self.client.list_blob_names(name_starts_with=str(folder)): yield item - def get_raw_blob_stream(self, blob_name) -> BytesIO: - blob_client = self.client.get_blob_client(blob_name) + def get_raw_blob_stream(self, blob_name: Path) -> BytesIO: + blob_client = self.client.get_blob_client(str(blob_name)) data = blob_client.download_blob().readall() return BytesIO(data) diff --git a/quixstreams/sources/community/blob_store/clients/base.py b/quixstreams/sources/community/blob_store/clients/base.py index 5f3a90458..3cc085ef1 100644 --- a/quixstreams/sources/community/blob_store/clients/base.py +++ b/quixstreams/sources/community/blob_store/clients/base.py @@ -1,6 +1,7 @@ from abc import abstractmethod from dataclasses import dataclass from io import BytesIO +from pathlib import Path from typing import Any, Iterable, Union __all__ = ("BlobClient",) @@ -10,10 +11,10 @@ class BlobClient: _client: Any _credentials: Union[dict, str] - location: str + location: Union[str, Path] - @abstractmethod @property + @abstractmethod def client(self): ... """ @@ -24,7 +25,7 @@ def client(self): ... """ @abstractmethod - def blob_finder(self, folder: str) -> Iterable[str]: ... + def blob_collector(self, folder: Path) -> Iterable[Path]: ... """ Find all blobs starting from a root folder. @@ -33,7 +34,12 @@ def blob_finder(self, folder: str) -> Iterable[str]: ... """ @abstractmethod - def get_raw_blob_stream(self, blob_path: str) -> BytesIO: ... + def get_root_folder_count(self, filepath: Path) -> int: ... + + """Counts the number of folders at filepath to assume partition counts.""" + + @abstractmethod + def get_raw_blob_stream(self, blob_path: Path) -> BytesIO: ... """ Obtain a specific blob in its raw form. diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index 91313dd07..64f8f921d 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -1,7 +1,7 @@ import logging from pathlib import Path from time import sleep -from typing import Generator, Optional, Union +from typing import BinaryIO, Generator, Optional, Union from quixstreams.models import Topic, TopicConfig from quixstreams.sources import Source @@ -107,18 +107,6 @@ def _replay_delay(self, current_timestamp: int): def _get_partition_count(self) -> int: return len([f for f in self._filepath.iterdir()]) - def default_topic(self) -> Topic: - """ - Uses the file structure to generate the desired partition count for the - internal topic. - :return: the original default topic, with updated partition count - """ - topic = super().default_topic() - topic.config = TopicConfig( - num_partitions=self._get_partition_count(), replication_factor=1 - ) - return topic - def _check_file_partition_number(self, file: Path): """ Checks whether the next file is the start of a new partition so the timestamp @@ -130,6 +118,12 @@ def _check_file_partition_number(self, file: Path): self._previous_partition = partition logger.debug(f"Beginning reading partition {partition}") + def _file_read(self, file: Union[Path, BinaryIO]) -> Generator[dict, None, None]: + yield from self._formatter.file_read(file) + + def _file_list(self) -> Generator[Path, None, None]: + yield from _file_finder(self._filepath) + def _produce(self, record: dict): kafka_msg = self._producer_topic.serialize( key=record["_key"], @@ -140,14 +134,26 @@ def _produce(self, record: dict): key=kafka_msg.key, value=kafka_msg.value, timestamp=kafka_msg.timestamp ) + def default_topic(self) -> Topic: + """ + Uses the file structure to generate the desired partition count for the + internal topic. + :return: the original default topic, with updated partition count + """ + topic = super().default_topic() + topic.config = TopicConfig( + num_partitions=self._get_partition_count(), replication_factor=1 + ) + return topic + def run(self): while self._running: - for file in _file_finder(self._filepath): + for file in self._file_list(): logger.info(f"Reading files from topic {self._filepath.name}") self._check_file_partition_number(file) - for record in self._formatter.file_read(file): - if self._as_replay: - self._replay_delay(record["_timestamp"]) + for record in self._file_read(file): + if self._as_replay and (timestamp := record.get("_timestamp")): + self._replay_delay(timestamp) self._produce(record) self.flush() return From 9045a000a29201e9c3c3b46766de903715f5e627 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 25 Nov 2024 20:40:11 -0500 Subject: [PATCH 04/24] a few more tweaks --- .../sources/community/blob_store/blob_source.py | 16 +++++----------- .../sources/community/blob_store/clients/aws.py | 2 +- .../community/blob_store/clients/azure.py | 8 ++++++-- .../sources/community/blob_store/clients/base.py | 2 +- 4 files changed, 13 insertions(+), 15 deletions(-) diff --git a/quixstreams/sources/community/blob_store/blob_source.py b/quixstreams/sources/community/blob_store/blob_source.py index 914a9e40a..d540937eb 100644 --- a/quixstreams/sources/community/blob_store/blob_source.py +++ b/quixstreams/sources/community/blob_store/blob_source.py @@ -19,34 +19,28 @@ def __init__( self, blob_client: BlobClient, blob_format: Union[FormatName, Format], + filepath: Union[str, Path], blob_compression: Optional[CompressionName] = None, - blob_folder: Optional[Union[str, Path]] = None, - blob_file: Optional[Union[str, Path]] = None, as_replay: bool = True, name: Optional[str] = None, shutdown_timeout: float = 10.0, ): self._client = blob_client - self._blob_file = Path(blob_file) if blob_file else None - self._blob_folder = Path(blob_folder) if blob_folder else None super().__init__( - filepath=self._client.location, + filepath=filepath, file_format=blob_format, file_compression=blob_compression, as_replay=as_replay, - name=name or self._client.location, + name=name or self._client.root_location, shutdown_timeout=shutdown_timeout, ) def _get_partition_count(self) -> int: - return self._client.get_root_folder_count(self._blob_folder) + return self._client.get_root_folder_count(self._filepath) def _file_read(self, file: Path) -> Generator[dict, None, None]: yield from super()._file_read(self._client.get_raw_blob_stream(file)) def _file_list(self) -> Generator[Path, None, None]: - if self._blob_file: - yield self._blob_file - else: - yield from self._client.blob_collector(self._blob_folder) + yield from self._client.blob_collector(self._filepath) diff --git a/quixstreams/sources/community/blob_store/clients/aws.py b/quixstreams/sources/community/blob_store/clients/aws.py index 6f747d8f9..6e58fc1d4 100644 --- a/quixstreams/sources/community/blob_store/clients/aws.py +++ b/quixstreams/sources/community/blob_store/clients/aws.py @@ -43,7 +43,7 @@ def __init__( to a locally hosted Kinesis. NOTE: can alternatively set the AWS_ENDPOINT_URL_KINESIS environment variable """ - self.location = aws_s3_bucket + self.root_location = aws_s3_bucket self._client: Optional[S3Client] = None self._credentials = { "region_name": aws_region, diff --git a/quixstreams/sources/community/blob_store/clients/azure.py b/quixstreams/sources/community/blob_store/clients/azure.py index 2830135a6..9626d2c7d 100644 --- a/quixstreams/sources/community/blob_store/clients/azure.py +++ b/quixstreams/sources/community/blob_store/clients/azure.py @@ -14,20 +14,24 @@ class AzureBlobClient(BlobClient): + def get_root_folder_count(self, filepath: Path) -> int: + # TODO: implement + ... + def __init__( self, connection_string: str, container: str, ): self._client: Optional[ContainerClient] = None - self.location = container + self.root_location = container self._credentials = connection_string @property def client(self): if not self._client: blob_client = BlobServiceClient.from_connection_string(self._credentials) - container_client = blob_client.get_container_client(self.location) + container_client = blob_client.get_container_client(self.root_location) self._client: ContainerClient = container_client return self._client diff --git a/quixstreams/sources/community/blob_store/clients/base.py b/quixstreams/sources/community/blob_store/clients/base.py index 3cc085ef1..f8e54688e 100644 --- a/quixstreams/sources/community/blob_store/clients/base.py +++ b/quixstreams/sources/community/blob_store/clients/base.py @@ -11,7 +11,7 @@ class BlobClient: _client: Any _credentials: Union[dict, str] - location: Union[str, Path] + root_location: Union[str, Path] @property @abstractmethod From 6fee37d29f8304f36185944165ba79355ac2336f Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Tue, 26 Nov 2024 15:39:56 -0500 Subject: [PATCH 05/24] move everything to file source --- .../sources/community/file/__init__.py | 1 + quixstreams/sources/community/file/file.py | 37 +++----- .../sources/community/file/formats/base.py | 3 +- .../community/file/origins/__init__.py | 4 + .../sources/community/file/origins/azure.py | 46 ++++++++++ .../sources/community/file/origins/base.py | 53 ++++++++++++ .../sources/community/file/origins/local.py | 36 ++++++++ .../sources/community/file/origins/s3.py | 85 +++++++++++++++++++ 8 files changed, 240 insertions(+), 25 deletions(-) create mode 100644 quixstreams/sources/community/file/origins/__init__.py create mode 100644 quixstreams/sources/community/file/origins/azure.py create mode 100644 quixstreams/sources/community/file/origins/base.py create mode 100644 quixstreams/sources/community/file/origins/local.py create mode 100644 quixstreams/sources/community/file/origins/s3.py diff --git a/quixstreams/sources/community/file/__init__.py b/quixstreams/sources/community/file/__init__.py index 9d6be93f9..73db23783 100644 --- a/quixstreams/sources/community/file/__init__.py +++ b/quixstreams/sources/community/file/__init__.py @@ -1,3 +1,4 @@ # ruff: noqa: F403 from .file import * from .formats import * +from .origins import * diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index 64f8f921d..ec0a623ef 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -1,13 +1,15 @@ import logging from pathlib import Path from time import sleep -from typing import BinaryIO, Generator, Optional, Union +from typing import Optional, Union from quixstreams.models import Topic, TopicConfig from quixstreams.sources import Source from .compressions import CompressionName from .formats import FORMATS, Format, FormatName +from .origins import LocalFileOrigin +from .origins.base import FileOrigin __all__ = ("FileSource",) @@ -63,7 +65,8 @@ class FileSource(Source): def __init__( self, filepath: Union[str, Path], - file_format: Union[Format, FormatName], + file_format: Union[Format, FormatName] = "json", + file_origin: Optional[FileOrigin] = LocalFileOrigin(), file_compression: Optional[CompressionName] = None, as_replay: bool = True, name: Optional[str] = None, @@ -75,6 +78,7 @@ def __init__( :param file_format: what format the message files are in (ex: json, parquet). Optionally, can provide a `Format` instance if more than file_compression is necessary to define (file_compression will then be ignored). + :param file_origin: a FileOrigin type (defaults to reading local files). :param file_compression: what compression is used on the given files, if any. :param as_replay: Produce the messages with the original time delay between them. Otherwise, produce the messages as fast as possible. @@ -84,6 +88,7 @@ def __init__( to gracefully shutdown """ self._filepath = Path(filepath) + self._origin = file_origin self._formatter = _get_formatter(file_format, file_compression) self._as_replay = as_replay self._previous_timestamp = None @@ -104,9 +109,6 @@ def _replay_delay(self, current_timestamp: int): sleep(time_diff) self._previous_timestamp = current_timestamp - def _get_partition_count(self) -> int: - return len([f for f in self._filepath.iterdir()]) - def _check_file_partition_number(self, file: Path): """ Checks whether the next file is the start of a new partition so the timestamp @@ -118,12 +120,6 @@ def _check_file_partition_number(self, file: Path): self._previous_partition = partition logger.debug(f"Beginning reading partition {partition}") - def _file_read(self, file: Union[Path, BinaryIO]) -> Generator[dict, None, None]: - yield from self._formatter.file_read(file) - - def _file_list(self) -> Generator[Path, None, None]: - yield from _file_finder(self._filepath) - def _produce(self, record: dict): kafka_msg = self._producer_topic.serialize( key=record["_key"], @@ -142,16 +138,19 @@ def default_topic(self) -> Topic: """ topic = super().default_topic() topic.config = TopicConfig( - num_partitions=self._get_partition_count(), replication_factor=1 + num_partitions=self._origin.get_root_folder_count(self._filepath), + replication_factor=1, ) return topic def run(self): while self._running: - for file in self._file_list(): - logger.info(f"Reading files from topic {self._filepath.name}") + logger.info(f"Reading files from topic {self._filepath.name}") + for file in self._origin.file_collector(self._filepath): + logger.debug(f"Reading file {file}") self._check_file_partition_number(file) - for record in self._file_read(file): + filestream = self._origin.get_raw_file_stream(file) + for record in self._formatter.read(filestream): if self._as_replay and (timestamp := record.get("_timestamp")): self._replay_delay(timestamp) self._produce(record) @@ -173,11 +172,3 @@ def _get_formatter( f"Allowed values: {allowed_formats}, " f"or an instance of a subclass of `Format`." ) - - -def _file_finder(filepath: Path) -> Generator[Path, None, None]: - if filepath.is_dir(): - for i in sorted(filepath.iterdir(), key=lambda x: x.name): - yield from _file_finder(i) - else: - yield filepath diff --git a/quixstreams/sources/community/file/formats/base.py b/quixstreams/sources/community/file/formats/base.py index 102ff9ba4..04b2a652f 100644 --- a/quixstreams/sources/community/file/formats/base.py +++ b/quixstreams/sources/community/file/formats/base.py @@ -57,7 +57,6 @@ def _decompress(self, filestream: BinaryIO) -> BinaryIO: def _open(self, file: Union[Path, BinaryIO]) -> BinaryIO: if isinstance(file, Path): with open(file, "rb") as f: - # yield for when no decompression is done yield self._decompress(f) else: yield self._decompress(file) @@ -65,6 +64,6 @@ def _open(self, file: Union[Path, BinaryIO]) -> BinaryIO: def _set_decompressor(self, extension_or_name: CompressionName): self._decompressor = COMPRESSION_MAPPER[extension_or_name]() - def file_read(self, file: Union[Path, BinaryIO]) -> Generator[dict, None, None]: + def read(self, file: Union[Path, BinaryIO]) -> Generator[dict, None, None]: with self._open(file) as filestream: yield from self.deserialize(filestream) diff --git a/quixstreams/sources/community/file/origins/__init__.py b/quixstreams/sources/community/file/origins/__init__.py new file mode 100644 index 000000000..dded4c541 --- /dev/null +++ b/quixstreams/sources/community/file/origins/__init__.py @@ -0,0 +1,4 @@ +# ruff: noqa: F403 +from .azure import * +from .local import * +from .s3 import * diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py new file mode 100644 index 000000000..93859a7c1 --- /dev/null +++ b/quixstreams/sources/community/file/origins/azure.py @@ -0,0 +1,46 @@ +from io import BytesIO +from pathlib import Path +from typing import Generator, Optional + +from .base import FileOrigin + +try: + from azure.storage.blob import BlobServiceClient + from azure.storage.blob._container_client import ContainerClient +except Exception: + raise + +__all__ = ("AzureFileOrigin",) + + +class AzureFileOrigin(FileOrigin): + def get_root_folder_count(self, filepath: Path) -> int: + # TODO: implement + ... + + def __init__( + self, + connection_string: str, + container: str, + ): + self._client: Optional[ContainerClient] = None + self.root_location = container + self._credentials = connection_string + + @property + def client(self): + if not self._client: + blob_client = BlobServiceClient.from_connection_string(self._credentials) + container_client = blob_client.get_container_client(self.root_location) + self._client: ContainerClient = container_client + return self._client + + def file_collector(self, folder: Path) -> Generator[str, None, None]: + # TODO: Recursively navigate folders. + for item in self.client.list_blob_names(name_starts_with=str(folder)): + yield item + + def get_raw_file_stream(self, blob_name: Path) -> BytesIO: + blob_client = self.client.get_blob_client(str(blob_name)) + data = blob_client.download_blob().readall() + return BytesIO(data) diff --git a/quixstreams/sources/community/file/origins/base.py b/quixstreams/sources/community/file/origins/base.py new file mode 100644 index 000000000..8deba88fe --- /dev/null +++ b/quixstreams/sources/community/file/origins/base.py @@ -0,0 +1,53 @@ +from abc import abstractmethod +from dataclasses import dataclass +from pathlib import Path +from typing import Any, BinaryIO, Iterable, Union + +__all__ = ("FileOrigin",) + + +@dataclass +class FileOrigin: + """ + An interface for interacting with a file-based client. + + Provides methods for navigating folders and retrieving/opening raw files. + """ + + _client: Any + _credentials: Union[dict, str] + root_location: Union[str, Path] + + @property + @abstractmethod + def client(self): ... + + """ + Set _client here. + + Circumvents pickling issues with multiprocessing by init-ing the client in a + later step (when Application.run() is called). + """ + + @abstractmethod + def file_collector(self, filepath: Path) -> Iterable[Path]: ... + + """ + Find all blobs starting from a root folder. + + Each item in the iterable should be a filepath resolvable by `get_raw_file_stream`. + """ + + @abstractmethod + def get_root_folder_count(self, folder: Path) -> int: ... + + """Counts the number of folders at filepath to assume partition counts.""" + + @abstractmethod + def get_raw_file_stream(self, filepath: Path) -> BinaryIO: ... + + """ + Obtain a file and return it as an (open) filestream. + + Result should be ready for deserialization (and/or decompression). + """ diff --git a/quixstreams/sources/community/file/origins/local.py b/quixstreams/sources/community/file/origins/local.py new file mode 100644 index 000000000..77d2a8aed --- /dev/null +++ b/quixstreams/sources/community/file/origins/local.py @@ -0,0 +1,36 @@ +from io import BytesIO +from pathlib import Path +from typing import Generator, Optional + +from .base import FileOrigin + +__all__ = ("LocalFileOrigin",) + + +class LocalFileOrigin(FileOrigin): + def __init__( + self, + ): + self._client = None + self._credentials = {} + self.root_location = "/" + + @property + def client(self): + return + + def file_collector( + self, filepath: Optional[Path] = None + ) -> Generator[Path, None, None]: + if filepath.is_dir(): + for i in sorted(filepath.iterdir(), key=lambda x: x.name): + yield from self.file_collector(i) + else: + yield filepath + + def get_root_folder_count(self, folder: Path) -> int: + return len([f for f in folder.iterdir()]) + + def get_raw_file_stream(self, filepath: Path) -> BytesIO: + with open(filepath, "rb") as f: + return BytesIO(f.read()) diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py new file mode 100644 index 000000000..d7bf49b94 --- /dev/null +++ b/quixstreams/sources/community/file/origins/s3.py @@ -0,0 +1,85 @@ +import logging +from io import BytesIO +from os import getenv +from pathlib import Path +from typing import Generator, Optional, Union + +from .base import FileOrigin + +try: + from boto3 import client as boto_client + from mypy_boto3_s3 import S3Client +except Exception: + raise + + +logger = logging.getLogger(__name__) + +__all__ = ("S3FileOrigin",) + + +class S3FileOrigin(FileOrigin): + def __init__( + self, + aws_s3_bucket: str, + aws_region: Optional[str] = getenv("AWS_REGION"), + aws_access_key_id: Optional[str] = getenv("AWS_ACCESS_KEY_ID"), + aws_secret_access_key: Optional[str] = getenv("AWS_SECRET_ACCESS_KEY"), + aws_endpoint_url: Optional[str] = getenv("AWS_ENDPOINT_URL_S3"), + aws_session_token: Optional[str] = None, + ): + """ + Configure IcebergSink to work with AWS Glue. + + :param aws_s3_bucket: The S3 URI where the table data will be stored + (e.g., 's3://your-bucket/warehouse/'). + :param aws_region: The AWS region. + NOTE: can alternatively set the AWS_REGION environment variable + :param aws_access_key_id: the AWS access key ID. + NOTE: can alternatively set the AWS_ACCESS_KEY_ID environment variable + :param aws_secret_access_key: the AWS secret access key. + NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable + :param aws_endpoint_url: the endpoint URL to use; only required for connecting + to a locally hosted Kinesis. + NOTE: can alternatively set the AWS_ENDPOINT_URL_KINESIS environment variable + """ + self.root_location = aws_s3_bucket + self._client: Optional[S3Client] = None + self._credentials = { + "region_name": aws_region, + "aws_access_key_id": aws_access_key_id, + "aws_secret_access_key": aws_secret_access_key, + "aws_session_token": aws_session_token, + "endpoint_url": aws_endpoint_url, + } + + @property + def client(self): + if not self._client: + self._client: S3Client = boto_client("s3", **self._credentials) + return self._client + + def get_raw_file_stream(self, filepath: Path) -> BytesIO: + data = self.client.get_object(Bucket=self.root_location, Key=str(filepath))[ + "Body" + ].read() + return BytesIO(data) + + def get_root_folder_count(self, folder: Path) -> int: + resp = self.client.list_objects( + Bucket=self.root_location, Prefix=str(folder), Delimiter="/" + ) + self._client = None + return len(resp["CommonPrefixes"]) + + def file_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None]: + resp = self.client.list_objects( + Bucket=self.root_location, + Prefix=str(folder), + Delimiter="/", + ) + for folder in resp.get("CommonPrefixes", []): + yield from self.file_collector(folder["Prefix"]) + + for file in resp.get("Contents", []): + yield Path(file["Key"]) From 5dd5d98db883519be0f965d9caea525d495a884b Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Tue, 26 Nov 2024 15:40:41 -0500 Subject: [PATCH 06/24] remove previous blob_store addition --- .../sources/community/blob_store/__init__.py | 5 -- .../community/blob_store/blob_source.py | 46 ---------- .../community/blob_store/clients/__init__.py | 3 - .../community/blob_store/clients/aws.py | 85 ------------------- .../community/blob_store/clients/azure.py | 46 ---------- .../community/blob_store/clients/base.py | 48 ----------- 6 files changed, 233 deletions(-) delete mode 100644 quixstreams/sources/community/blob_store/__init__.py delete mode 100644 quixstreams/sources/community/blob_store/blob_source.py delete mode 100644 quixstreams/sources/community/blob_store/clients/__init__.py delete mode 100644 quixstreams/sources/community/blob_store/clients/aws.py delete mode 100644 quixstreams/sources/community/blob_store/clients/azure.py delete mode 100644 quixstreams/sources/community/blob_store/clients/base.py diff --git a/quixstreams/sources/community/blob_store/__init__.py b/quixstreams/sources/community/blob_store/__init__.py deleted file mode 100644 index 9466c031f..000000000 --- a/quixstreams/sources/community/blob_store/__init__.py +++ /dev/null @@ -1,5 +0,0 @@ -# ruff: noqa: F403 -from quixstreams.sources.community.file.formats import * - -from .blob_source import * -from .clients import * diff --git a/quixstreams/sources/community/blob_store/blob_source.py b/quixstreams/sources/community/blob_store/blob_source.py deleted file mode 100644 index d540937eb..000000000 --- a/quixstreams/sources/community/blob_store/blob_source.py +++ /dev/null @@ -1,46 +0,0 @@ -import logging -from pathlib import Path -from typing import Generator, Optional, Union - -from quixstreams.sources.community.file import FileSource -from quixstreams.sources.community.file.compressions import CompressionName -from quixstreams.sources.community.file.formats import Format, FormatName - -from .clients.base import BlobClient - -logger = logging.getLogger(__name__) - - -__all__ = ("BlobFileSource",) - - -class BlobFileSource(FileSource): - def __init__( - self, - blob_client: BlobClient, - blob_format: Union[FormatName, Format], - filepath: Union[str, Path], - blob_compression: Optional[CompressionName] = None, - as_replay: bool = True, - name: Optional[str] = None, - shutdown_timeout: float = 10.0, - ): - self._client = blob_client - - super().__init__( - filepath=filepath, - file_format=blob_format, - file_compression=blob_compression, - as_replay=as_replay, - name=name or self._client.root_location, - shutdown_timeout=shutdown_timeout, - ) - - def _get_partition_count(self) -> int: - return self._client.get_root_folder_count(self._filepath) - - def _file_read(self, file: Path) -> Generator[dict, None, None]: - yield from super()._file_read(self._client.get_raw_blob_stream(file)) - - def _file_list(self) -> Generator[Path, None, None]: - yield from self._client.blob_collector(self._filepath) diff --git a/quixstreams/sources/community/blob_store/clients/__init__.py b/quixstreams/sources/community/blob_store/clients/__init__.py deleted file mode 100644 index 1bac3f560..000000000 --- a/quixstreams/sources/community/blob_store/clients/__init__.py +++ /dev/null @@ -1,3 +0,0 @@ -# ruff: noqa: F403 -from .aws import * -from .azure import * diff --git a/quixstreams/sources/community/blob_store/clients/aws.py b/quixstreams/sources/community/blob_store/clients/aws.py deleted file mode 100644 index 6e58fc1d4..000000000 --- a/quixstreams/sources/community/blob_store/clients/aws.py +++ /dev/null @@ -1,85 +0,0 @@ -import logging -from io import BytesIO -from os import getenv -from pathlib import Path -from typing import Generator, Optional, Union - -from .base import BlobClient - -try: - from boto3 import client as boto_client - from mypy_boto3_s3 import S3Client -except Exception: - raise - - -logger = logging.getLogger(__name__) - -__all__ = ("AwsS3BlobClient",) - - -class AwsS3BlobClient(BlobClient): - def __init__( - self, - aws_s3_bucket: str, - aws_region: Optional[str] = getenv("AWS_REGION"), - aws_access_key_id: Optional[str] = getenv("AWS_ACCESS_KEY_ID"), - aws_secret_access_key: Optional[str] = getenv("AWS_SECRET_ACCESS_KEY"), - aws_endpoint_url: Optional[str] = getenv("AWS_ENDPOINT_URL_S3"), - aws_session_token: Optional[str] = None, - ): - """ - Configure IcebergSink to work with AWS Glue. - - :param aws_s3_bucket: The S3 URI where the table data will be stored - (e.g., 's3://your-bucket/warehouse/'). - :param aws_region: The AWS region. - NOTE: can alternatively set the AWS_REGION environment variable - :param aws_access_key_id: the AWS access key ID. - NOTE: can alternatively set the AWS_ACCESS_KEY_ID environment variable - :param aws_secret_access_key: the AWS secret access key. - NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable - :param aws_endpoint_url: the endpoint URL to use; only required for connecting - to a locally hosted Kinesis. - NOTE: can alternatively set the AWS_ENDPOINT_URL_KINESIS environment variable - """ - self.root_location = aws_s3_bucket - self._client: Optional[S3Client] = None - self._credentials = { - "region_name": aws_region, - "aws_access_key_id": aws_access_key_id, - "aws_secret_access_key": aws_secret_access_key, - "aws_session_token": aws_session_token, - "endpoint_url": aws_endpoint_url, - } - - @property - def client(self): - if not self._client: - self._client: S3Client = boto_client("s3", **self._credentials) - return self._client - - def get_raw_blob_stream(self, blob_path: Path) -> BytesIO: - data = self.client.get_object(Bucket=self.location, Key=str(blob_path))[ - "Body" - ].read() - return BytesIO(data) - - def get_root_folder_count(self, folder: Path) -> int: - resp = self.client.list_objects( - Bucket=self.location, Prefix=str(folder), Delimiter="/" - ) - self._client = None - return len(resp["CommonPrefixes"]) - - def blob_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None]: - resp = self.client.list_objects( - Bucket=self.location, - Prefix=str(folder), - Delimiter="/", - ) - for folder in resp.get("CommonPrefixes", []): - yield from self.blob_collector(folder["Prefix"]) - - for file in resp.get("Contents", []): - yield Path(file["Key"]) diff --git a/quixstreams/sources/community/blob_store/clients/azure.py b/quixstreams/sources/community/blob_store/clients/azure.py deleted file mode 100644 index 9626d2c7d..000000000 --- a/quixstreams/sources/community/blob_store/clients/azure.py +++ /dev/null @@ -1,46 +0,0 @@ -from io import BytesIO -from pathlib import Path -from typing import Generator, Optional - -from .base import BlobClient - -try: - from azure.storage.blob import BlobServiceClient - from azure.storage.blob._container_client import ContainerClient -except Exception: - raise - -__all__ = ("AzureBlobClient",) - - -class AzureBlobClient(BlobClient): - def get_root_folder_count(self, filepath: Path) -> int: - # TODO: implement - ... - - def __init__( - self, - connection_string: str, - container: str, - ): - self._client: Optional[ContainerClient] = None - self.root_location = container - self._credentials = connection_string - - @property - def client(self): - if not self._client: - blob_client = BlobServiceClient.from_connection_string(self._credentials) - container_client = blob_client.get_container_client(self.root_location) - self._client: ContainerClient = container_client - return self._client - - def blob_collector(self, folder: Path) -> Generator[str, None, None]: - # TODO: Recursively navigate folders. - for item in self.client.list_blob_names(name_starts_with=str(folder)): - yield item - - def get_raw_blob_stream(self, blob_name: Path) -> BytesIO: - blob_client = self.client.get_blob_client(str(blob_name)) - data = blob_client.download_blob().readall() - return BytesIO(data) diff --git a/quixstreams/sources/community/blob_store/clients/base.py b/quixstreams/sources/community/blob_store/clients/base.py deleted file mode 100644 index f8e54688e..000000000 --- a/quixstreams/sources/community/blob_store/clients/base.py +++ /dev/null @@ -1,48 +0,0 @@ -from abc import abstractmethod -from dataclasses import dataclass -from io import BytesIO -from pathlib import Path -from typing import Any, Iterable, Union - -__all__ = ("BlobClient",) - - -@dataclass -class BlobClient: - _client: Any - _credentials: Union[dict, str] - root_location: Union[str, Path] - - @property - @abstractmethod - def client(self): ... - - """ - Set _client here. - - Circumvents pickling issues with multiprocessing by init'ing the client in a - later step (when Application.run() is called). - """ - - @abstractmethod - def blob_collector(self, folder: Path) -> Iterable[Path]: ... - - """ - Find all blobs starting from a root folder. - - Each item in the iterable should be a filepath resolvable by `get_raw_blob_stream`. - """ - - @abstractmethod - def get_root_folder_count(self, filepath: Path) -> int: ... - - """Counts the number of folders at filepath to assume partition counts.""" - - @abstractmethod - def get_raw_blob_stream(self, blob_path: Path) -> BytesIO: ... - - """ - Obtain a specific blob in its raw form. - - Result should be ready for deserialization (and/or decompression). - """ From dae3fc622c9586784cc762123a9d8416ae96360d Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Tue, 26 Nov 2024 23:00:58 -0500 Subject: [PATCH 07/24] clean up some things, add pagination handling to azure --- quixstreams/sources/community/file/file.py | 2 +- .../sources/community/file/formats/base.py | 17 +++------------ .../sources/community/file/origins/azure.py | 21 ++++++++++++------- .../sources/community/file/origins/base.py | 2 +- .../sources/community/file/origins/local.py | 2 +- .../sources/community/file/origins/s3.py | 17 +++++++-------- 6 files changed, 27 insertions(+), 34 deletions(-) diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index ec0a623ef..6e7ebabc1 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -138,7 +138,7 @@ def default_topic(self) -> Topic: """ topic = super().default_topic() topic.config = TopicConfig( - num_partitions=self._origin.get_root_folder_count(self._filepath), + num_partitions=self._origin.get_folder_count(self._filepath), replication_factor=1, ) return topic diff --git a/quixstreams/sources/community/file/formats/base.py b/quixstreams/sources/community/file/formats/base.py index 04b2a652f..d18332d39 100644 --- a/quixstreams/sources/community/file/formats/base.py +++ b/quixstreams/sources/community/file/formats/base.py @@ -1,8 +1,6 @@ -import contextlib from abc import ABC, abstractmethod from io import BytesIO -from pathlib import Path -from typing import BinaryIO, Generator, Iterable, Literal, Optional, Union +from typing import BinaryIO, Generator, Iterable, Literal, Optional from ..compressions import COMPRESSION_MAPPER, CompressionName, Decompressor @@ -53,17 +51,8 @@ def _decompress(self, filestream: BinaryIO) -> BinaryIO: return filestream return BytesIO(self._decompressor.decompress(filestream)) - @contextlib.contextmanager - def _open(self, file: Union[Path, BinaryIO]) -> BinaryIO: - if isinstance(file, Path): - with open(file, "rb") as f: - yield self._decompress(f) - else: - yield self._decompress(file) - def _set_decompressor(self, extension_or_name: CompressionName): self._decompressor = COMPRESSION_MAPPER[extension_or_name]() - def read(self, file: Union[Path, BinaryIO]) -> Generator[dict, None, None]: - with self._open(file) as filestream: - yield from self.deserialize(filestream) + def read(self, filestream: BinaryIO) -> Generator[dict, None, None]: + yield from self.deserialize(self._decompress(filestream)) diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py index 93859a7c1..a83e8afc3 100644 --- a/quixstreams/sources/community/file/origins/azure.py +++ b/quixstreams/sources/community/file/origins/azure.py @@ -7,16 +7,20 @@ try: from azure.storage.blob import BlobServiceClient from azure.storage.blob._container_client import ContainerClient -except Exception: - raise +except ImportError as exc: + raise ImportError( + f"Package {exc.name} is missing: " + 'run "pip install quixstreams[azure]" to use AzureFileOrigin' + ) from exc __all__ = ("AzureFileOrigin",) class AzureFileOrigin(FileOrigin): - def get_root_folder_count(self, filepath: Path) -> int: - # TODO: implement - ... + def get_folder_count(self, filepath: Path) -> int: + data = self.client.list_blobs(name_starts_with=str(filepath)) + folders = [f for page in data.by_page() for f in page.get("blob_prefixes", [])] + return len(folders) def __init__( self, @@ -36,9 +40,10 @@ def client(self): return self._client def file_collector(self, folder: Path) -> Generator[str, None, None]: - # TODO: Recursively navigate folders. - for item in self.client.list_blob_names(name_starts_with=str(folder)): - yield item + data = self.client.list_blob_names(name_starts_with=str(folder)) + for page in data.by_page(): + for item in page: + yield item def get_raw_file_stream(self, blob_name: Path) -> BytesIO: blob_client = self.client.get_blob_client(str(blob_name)) diff --git a/quixstreams/sources/community/file/origins/base.py b/quixstreams/sources/community/file/origins/base.py index 8deba88fe..84974d531 100644 --- a/quixstreams/sources/community/file/origins/base.py +++ b/quixstreams/sources/community/file/origins/base.py @@ -39,7 +39,7 @@ def file_collector(self, filepath: Path) -> Iterable[Path]: ... """ @abstractmethod - def get_root_folder_count(self, folder: Path) -> int: ... + def get_folder_count(self, folder: Path) -> int: ... """Counts the number of folders at filepath to assume partition counts.""" diff --git a/quixstreams/sources/community/file/origins/local.py b/quixstreams/sources/community/file/origins/local.py index 77d2a8aed..3ad09f746 100644 --- a/quixstreams/sources/community/file/origins/local.py +++ b/quixstreams/sources/community/file/origins/local.py @@ -28,7 +28,7 @@ def file_collector( else: yield filepath - def get_root_folder_count(self, folder: Path) -> int: + def get_folder_count(self, folder: Path) -> int: return len([f for f in folder.iterdir()]) def get_raw_file_stream(self, filepath: Path) -> BytesIO: diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index d7bf49b94..bf9433d32 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -9,9 +9,11 @@ try: from boto3 import client as boto_client from mypy_boto3_s3 import S3Client -except Exception: - raise - +except ImportError as exc: + raise ImportError( + f"Package {exc.name} is missing: " + 'run "pip install quixstreams[aws]" to use S3FileOrigin' + ) from exc logger = logging.getLogger(__name__) @@ -26,13 +28,11 @@ def __init__( aws_access_key_id: Optional[str] = getenv("AWS_ACCESS_KEY_ID"), aws_secret_access_key: Optional[str] = getenv("AWS_SECRET_ACCESS_KEY"), aws_endpoint_url: Optional[str] = getenv("AWS_ENDPOINT_URL_S3"), - aws_session_token: Optional[str] = None, ): """ Configure IcebergSink to work with AWS Glue. - :param aws_s3_bucket: The S3 URI where the table data will be stored - (e.g., 's3://your-bucket/warehouse/'). + :param aws_s3_bucket: The S3 URI with bucket name (e.g., 's3://your-bucket'). :param aws_region: The AWS region. NOTE: can alternatively set the AWS_REGION environment variable :param aws_access_key_id: the AWS access key ID. @@ -41,7 +41,7 @@ def __init__( NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable :param aws_endpoint_url: the endpoint URL to use; only required for connecting to a locally hosted Kinesis. - NOTE: can alternatively set the AWS_ENDPOINT_URL_KINESIS environment variable + NOTE: can alternatively set the AWS_ENDPOINT_URL_S3 environment variable """ self.root_location = aws_s3_bucket self._client: Optional[S3Client] = None @@ -49,7 +49,6 @@ def __init__( "region_name": aws_region, "aws_access_key_id": aws_access_key_id, "aws_secret_access_key": aws_secret_access_key, - "aws_session_token": aws_session_token, "endpoint_url": aws_endpoint_url, } @@ -65,7 +64,7 @@ def get_raw_file_stream(self, filepath: Path) -> BytesIO: ].read() return BytesIO(data) - def get_root_folder_count(self, folder: Path) -> int: + def get_folder_count(self, folder: Path) -> int: resp = self.client.list_objects( Bucket=self.root_location, Prefix=str(folder), Delimiter="/" ) From 211b3cd06393f6435de876c684322498b262d176 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 00:52:09 -0500 Subject: [PATCH 08/24] add docs, rename some stuff --- docs/connectors/sources/aws-s3-file-source.md | 158 ++++++++++++++++++ .../{file-source.md => local-file-source.md} | 0 quixstreams/sources/community/file/file.py | 18 +- .../sources/community/file/origins/s3.py | 2 +- 4 files changed, 174 insertions(+), 4 deletions(-) create mode 100644 docs/connectors/sources/aws-s3-file-source.md rename docs/connectors/sources/{file-source.md => local-file-source.md} (100%) diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/aws-s3-file-source.md new file mode 100644 index 000000000..2d8747c7e --- /dev/null +++ b/docs/connectors/sources/aws-s3-file-source.md @@ -0,0 +1,158 @@ +# AWS S3 File Source + +!!! info + + This is a **Community** connector. Test it before using in production. + + To learn more about differences between Core and Community connectors, see the [Community and Core Connectors](../community-and-core.md) page. + +This source reads records from files located in an Amazon S3 bucket path and produces +them as messages to a kafka topic using any desired `StreamingDataFrame`-based transformations. + +The resulting messages can be produced in "replay" mode, where the time between record +producing is matched as close as possible to the original. (per topic partition only). + +The `FileSource` connector is generally intended to be used alongside the related +[`FileSink`](../sinks/file-sink.md) (in terms of expected file and data formatting). + + +## How To Install + +Install Quix Streams with the following optional dependencies: + +```bash +pip install quixstreams[aws] +``` + +## How It Works + +`FileSource` steps through each folder within the provided path and dumps each record +contained in each file as a message to a Kafka topic. Folders are navigated in +lexicographical order. + +Records are read in a streaming fashion and committed after every file, offering +[at-least-once guarantees](#processingdelivery-guarantees). + +It can handle one given file type (ex. JSONlines or Parquet) at a time, and also +supports file decompression. + +You can learn more details about the [expected kafka message format](#message-data-formatschema) below. + +## How To Use + +S3 File Source is just a special configuration of the `FileSource` connector. + +Simply provide it an `S3FileOrigin` (`FileSource(origin=)`). + +Then, hand the configured `FileSource` to your SDF (`app.dataframe(source=)`). + +For more details around various settings, see [configuration](#configuration). + +```python +from quixstreams import Application +from quixstreams.sources.community.file import FileSource +from quixstreams.sources.community.file.origins import S3FileOrigin + +app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") + +file_origin = S3FileOrigin( + aws_s3_bucket="", + aws_access_key_id="", + aws_secret_access_key="", + aws_region="", +) +source = FileSource( + filepath="/path/to/your/topic_folder", + file_origin=file_origin, + file_format="json", + file_compression="gzip", +) +sdf = app.dataframe(source=source).print(metadata=True) +# YOUR LOGIC HERE! + +if __name__ == "__main__": + app.run() +``` + +## Configuration + +Here are some important configurations to be aware of (see [File Source API](../../api-reference/sources.md#filesource) for all parameters). + +### Required: + +`S3FileOrigin`: + +- `aws_s3_bucket`: The S3 URI with bucket name only (ex: 's3://your-bucket'). +- `aws_region`: AWS region (ex: us-east-1). + **Note**: can alternatively set the `AWS_REGION` environment variable. +- `aws_access_key_id`: AWS User key ID. + **Note**: can alternatively set the `AWS_ACCESS_KEY_ID` environment variable. +- `aws_secret_access_key`: AWS secret key. + **Note**: can alternatively set the `AWS_SECRET_ACCESS_KEY` environment variable. + + +`FileSource`: + +- `filepath`: a filepath to recursively read through (exclude bucket name). + **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). +- `file_origin`: An `S3FileOrigin` instance. + + +### Optional: + +`FileSource`: + +- `file_format`: what format the message files are in (ex: `"json"`, `"parquet"`). + **Advanced**: can optionally provide a `Format` instance (`file_compression` will then be ignored). + **Default**: `"json"` +- `file_compression`: what compression is used on the given files, if any (ex: `"gzip"`) + **Default**: `None` +- `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. + **Note**: Time delay will only be accurate _per partition_, NOT overall. + **Default**: `True` + +## Message Data Format/Schema + +The expected data schema produced by `Application` is largely dependent on the chosen +file format. + +For easiest use (especially alongside [`FileSink`](../sinks/file-sink.md)), you can follow these patterns: + +- for row-based formats (like JSON), the expected data should have records +with the following fields, where value is the entirety of the message value, +ideally as a JSON-deserializable item: + - `_key` + - `_value` + - `_timestamp` + +- for columnar formats (like Parquet), they do not expect an explicit `value` +field; instead all columns should be included individually while including `_key` and `_timestamp`: + - `_key` + - `_timestamp` + - `field_a` + - `field_b` + etc... + +### Custom Schemas (Advanced) + +Alternatively, custom schemas can be defined by supplying a configured `Format` +(ex: `JsonFormat`) to `FileSource(file_format=)`. + +Formats can be imported from `quixstreams.sources.community.file.formats`. + +## Processing/Delivery Guarantees + +This Source offers "at-least-once" guarantees with message delivery: messages are +guaranteed to be committed when a file is finished processing. + +However, it does not save any state/position: an unhandled exception will cause the +`Application` to fail, and rerunning the `Application` will begin processing from the +beginning (reproducing all previously processed messages). + +## Topic + +The default topic will have a partition count that reflects the partition count found +within the provided topic's folder structure. + +The default topic name the Application dumps to is based on the last folder name of +the `FileSource` `filepath` as: `source__`. diff --git a/docs/connectors/sources/file-source.md b/docs/connectors/sources/local-file-source.md similarity index 100% rename from docs/connectors/sources/file-source.md rename to docs/connectors/sources/local-file-source.md diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index 6e7ebabc1..ce6e2a5b4 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -18,8 +18,10 @@ class FileSource(Source): """ - Ingest a set of local files into kafka by iterating through the provided folder and - processing all nested files within it. + Ingest a set of files from a desired origin into kafka by iterating through the + provided folder and processing all nested files within it. + + Origins include a local filestore, AWS S3, or Microsoft Azure. Expects folder and file structures as generated by the related FileSink connector: @@ -48,14 +50,24 @@ class FileSource(Source): ```python from quixstreams import Application from quixstreams.sources.community.file import FileSource + from quixstreams.sources.community.file.origins import S3FileOrigin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") + + file_origin = S3FileOrigin( + aws_s3_bucket="", + aws_access_key_id="", + aws_secret_access_key="", + aws_region="", + ) source = FileSource( - filepath="/path/to/my/topic_folder", + filepath="/path/to/your/topic_folder", + file_origin=file_origin, file_format="json", file_compression="gzip", ) sdf = app.dataframe(source=source).print(metadata=True) + # YOUR LOGIC HERE! if __name__ == "__main__": app.run() diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index bf9433d32..0a266437b 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -32,7 +32,7 @@ def __init__( """ Configure IcebergSink to work with AWS Glue. - :param aws_s3_bucket: The S3 URI with bucket name (e.g., 's3://your-bucket'). + :param aws_s3_bucket: The S3 URI with bucket name only (ex: 's3://your-bucket'). :param aws_region: The AWS region. NOTE: can alternatively set the AWS_REGION environment variable :param aws_access_key_id: the AWS access key ID. From e23fc576c437f39507866e16263f8a3dcda50b27 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 01:32:17 -0500 Subject: [PATCH 09/24] clean up schema doc section --- docs/connectors/sources/aws-s3-file-source.md | 41 +++++++++++++------ 1 file changed, 29 insertions(+), 12 deletions(-) diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/aws-s3-file-source.md index 2d8747c7e..b74308def 100644 --- a/docs/connectors/sources/aws-s3-file-source.md +++ b/docs/connectors/sources/aws-s3-file-source.md @@ -12,9 +12,6 @@ them as messages to a kafka topic using any desired `StreamingDataFrame`-based t The resulting messages can be produced in "replay" mode, where the time between record producing is matched as close as possible to the original. (per topic partition only). -The `FileSource` connector is generally intended to be used alongside the related -[`FileSink`](../sinks/file-sink.md) (in terms of expected file and data formatting). - ## How To Install @@ -113,30 +110,50 @@ Here are some important configurations to be aware of (see [File Source API](../ ## Message Data Format/Schema -The expected data schema produced by `Application` is largely dependent on the chosen +The expected file schema largely depends on the chosen file format. -For easiest use (especially alongside [`FileSink`](../sinks/file-sink.md)), you can follow these patterns: +For easiest use (especially alongside [`FileSink`](../sinks/file-sink.md)), +you can follow these patterns: + +### Row-based Formats (ex: JSON) + +Files should have records with the following fields, with `_value` being a +JSON-deserializable item: -- for row-based formats (like JSON), the expected data should have records -with the following fields, where value is the entirety of the message value, -ideally as a JSON-deserializable item: - `_key` - `_value` - `_timestamp` -- for columnar formats (like Parquet), they do not expect an explicit `value` -field; instead all columns should be included individually while including `_key` and `_timestamp`: + +This will result in the following Kafka message format for `Application`: + +- Message `key` will be the record `_key` as `bytes`. +- Message `value` will be the record `_value` as a `json`/`dict` +- Message `timestamp` will be the record `_timestamp` (ms). + +### Columnar Formats (ex: Parquet) +These do not expect an explicit `value` field; instead all columns should be included +individually while including `_key` and `_timestamp`: + - `_key` - `_timestamp` - `field_a` - `field_b` etc... + +This will result in the following Kafka message format for `Application`: + +- Message `key` will be the record `_key` as `bytes`. +- Message `value` will be every record field except `_key` and `_timestamp` packed as a `json`/`dict` +- Message `timestamp` will be the record `_timestamp` (ms). + + ### Custom Schemas (Advanced) -Alternatively, custom schemas can be defined by supplying a configured `Format` -(ex: `JsonFormat`) to `FileSource(file_format=)`. +If the original files are not formatted as expected, custom loaders can be configured +on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(file_format=)`. Formats can be imported from `quixstreams.sources.community.file.formats`. From 71792dfbda28ad21f6156a156cd0377ae49797f5 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 01:43:50 -0500 Subject: [PATCH 10/24] clean up compressor --- .../community/file/compressions/base.py | 18 ++---------------- .../community/file/compressions/gzip.py | 2 +- 2 files changed, 3 insertions(+), 17 deletions(-) diff --git a/quixstreams/sources/community/file/compressions/base.py b/quixstreams/sources/community/file/compressions/base.py index 0f302d958..c2c772259 100644 --- a/quixstreams/sources/community/file/compressions/base.py +++ b/quixstreams/sources/community/file/compressions/base.py @@ -1,7 +1,5 @@ -import contextlib from abc import ABC, abstractmethod -from pathlib import Path -from typing import BinaryIO, Literal, Union +from typing import BinaryIO, Literal __all__ = ( "Decompressor", @@ -14,16 +12,4 @@ class Decompressor(ABC): @abstractmethod - def _decompress(self, filestream: BinaryIO) -> bytes: ... - - @contextlib.contextmanager - def _open(self, file: Union[Path, BinaryIO]) -> BinaryIO: - if isinstance(file, Path): - with open(file, "rb") as f: - yield f - else: - yield file - - def decompress(self, file: Union[Path, BinaryIO]) -> bytes: - with self._open(file) as filestream: - return self._decompress(filestream) + def decompress(self, filestream: BinaryIO) -> bytes: ... diff --git a/quixstreams/sources/community/file/compressions/gzip.py b/quixstreams/sources/community/file/compressions/gzip.py index 50cda2d82..2aaf67442 100644 --- a/quixstreams/sources/community/file/compressions/gzip.py +++ b/quixstreams/sources/community/file/compressions/gzip.py @@ -10,5 +10,5 @@ class GZipDecompressor(Decompressor): def __init__(self): self._decompressor = decompress - def _decompress(self, filestream: BinaryIO) -> bytes: + def decompress(self, filestream: BinaryIO) -> bytes: return self._decompressor(filestream.read()) From 4c3e6c38c9acb7039560c1b75630bfa2febb38d2 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 01:46:42 -0500 Subject: [PATCH 11/24] tiny doc tweaks --- docs/connectors/sources/aws-s3-file-source.md | 4 ++-- quixstreams/sources/community/file/file.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/aws-s3-file-source.md index b74308def..bb741903d 100644 --- a/docs/connectors/sources/aws-s3-file-source.md +++ b/docs/connectors/sources/aws-s3-file-source.md @@ -6,7 +6,7 @@ To learn more about differences between Core and Community connectors, see the [Community and Core Connectors](../community-and-core.md) page. -This source reads records from files located in an Amazon S3 bucket path and produces +This source reads records from files located in an AWS S3 bucket path and produces them as messages to a kafka topic using any desired `StreamingDataFrame`-based transformations. The resulting messages can be produced in "replay" mode, where the time between record @@ -59,7 +59,7 @@ file_origin = S3FileOrigin( aws_region="", ) source = FileSource( - filepath="/path/to/your/topic_folder", + filepath="path/to/your/topic_folder/", file_origin=file_origin, file_format="json", file_compression="gzip", diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index ce6e2a5b4..1eb22c64a 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -61,7 +61,7 @@ class FileSource(Source): aws_region="", ) source = FileSource( - filepath="/path/to/your/topic_folder", + filepath="path/to/your/topic_folder/", file_origin=file_origin, file_format="json", file_compression="gzip", From 8cff599cbefe72d7b5c4f7d27a65e32c0da4cb29 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 12:28:19 -0500 Subject: [PATCH 12/24] changes based on code review --- docs/connectors/sources/aws-s3-file-source.md | 4 +-- quixstreams/sources/community/file/file.py | 16 +++++---- .../sources/community/file/origins/azure.py | 26 +++++++-------- .../sources/community/file/origins/base.py | 33 ++++++++----------- .../sources/community/file/origins/local.py | 15 ++++----- .../sources/community/file/origins/s3.py | 30 ++++++++--------- 6 files changed, 58 insertions(+), 66 deletions(-) diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/aws-s3-file-source.md index bb741903d..c9e782393 100644 --- a/docs/connectors/sources/aws-s3-file-source.md +++ b/docs/connectors/sources/aws-s3-file-source.md @@ -53,7 +53,7 @@ from quixstreams.sources.community.file.origins import S3FileOrigin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") file_origin = S3FileOrigin( - aws_s3_bucket="", + aws_s3_bucket="", aws_access_key_id="", aws_secret_access_key="", aws_region="", @@ -79,7 +79,7 @@ Here are some important configurations to be aware of (see [File Source API](../ `S3FileOrigin`: -- `aws_s3_bucket`: The S3 URI with bucket name only (ex: 's3://your-bucket'). +- `aws_s3_bucket`: The S3 bucket name only (ex: `"your-bucket"`). - `aws_region`: AWS region (ex: us-east-1). **Note**: can alternatively set the `AWS_REGION` environment variable. - `aws_access_key_id`: AWS User key ID. diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index 1eb22c64a..0cf79e303 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -8,8 +8,8 @@ from .compressions import CompressionName from .formats import FORMATS, Format, FormatName -from .origins import LocalFileOrigin -from .origins.base import FileOrigin +from .origins import LocalOrigin +from .origins.base import Origin __all__ = ("FileSource",) @@ -18,11 +18,13 @@ class FileSource(Source): """ - Ingest a set of files from a desired origin into kafka by iterating through the + Ingest a set of files from a desired origin into Kafka by iterating through the provided folder and processing all nested files within it. Origins include a local filestore, AWS S3, or Microsoft Azure. + FileSource defaults to a local filestore (LocalOrigin) + JSON format. + Expects folder and file structures as generated by the related FileSink connector: ``` @@ -50,11 +52,11 @@ class FileSource(Source): ```python from quixstreams import Application from quixstreams.sources.community.file import FileSource - from quixstreams.sources.community.file.origins import S3FileOrigin + from quixstreams.sources.community.file.origins import S3Origin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") - file_origin = S3FileOrigin( + file_origin = S3Origin( aws_s3_bucket="", aws_access_key_id="", aws_secret_access_key="", @@ -78,7 +80,7 @@ def __init__( self, filepath: Union[str, Path], file_format: Union[Format, FormatName] = "json", - file_origin: Optional[FileOrigin] = LocalFileOrigin(), + file_origin: Origin = LocalOrigin(), file_compression: Optional[CompressionName] = None, as_replay: bool = True, name: Optional[str] = None, @@ -90,7 +92,7 @@ def __init__( :param file_format: what format the message files are in (ex: json, parquet). Optionally, can provide a `Format` instance if more than file_compression is necessary to define (file_compression will then be ignored). - :param file_origin: a FileOrigin type (defaults to reading local files). + :param file_origin: an Origin type (defaults to reading local files). :param file_compression: what compression is used on the given files, if any. :param as_replay: Produce the messages with the original time delay between them. Otherwise, produce the messages as fast as possible. diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py index a83e8afc3..2e99143cc 100644 --- a/quixstreams/sources/community/file/origins/azure.py +++ b/quixstreams/sources/community/file/origins/azure.py @@ -2,7 +2,7 @@ from pathlib import Path from typing import Generator, Optional -from .base import FileOrigin +from .base import ExternalOrigin try: from azure.storage.blob import BlobServiceClient @@ -10,15 +10,15 @@ except ImportError as exc: raise ImportError( f"Package {exc.name} is missing: " - 'run "pip install quixstreams[azure]" to use AzureFileOrigin' + 'run "pip install quixstreams[azure]" to use AzureOrigin' ) from exc -__all__ = ("AzureFileOrigin",) +__all__ = ("AzureOrigin",) -class AzureFileOrigin(FileOrigin): +class AzureOrigin(ExternalOrigin): def get_folder_count(self, filepath: Path) -> int: - data = self.client.list_blobs(name_starts_with=str(filepath)) + data = self._client.list_blobs(name_starts_with=str(filepath)) folders = [f for page in data.by_page() for f in page.get("blob_prefixes", [])] return len(folders) @@ -27,25 +27,23 @@ def __init__( connection_string: str, container: str, ): - self._client: Optional[ContainerClient] = None + self._client: Optional[ContainerClient] = self._get_client(connection_string) self.root_location = container - self._credentials = connection_string - @property - def client(self): + def _get_client(self, auth: str): if not self._client: - blob_client = BlobServiceClient.from_connection_string(self._credentials) + blob_client = BlobServiceClient.from_connection_string(auth) container_client = blob_client.get_container_client(self.root_location) self._client: ContainerClient = container_client return self._client - def file_collector(self, folder: Path) -> Generator[str, None, None]: - data = self.client.list_blob_names(name_starts_with=str(folder)) + def file_collector(self, folder: Path) -> Generator[Path, None, None]: + data = self._client.list_blob_names(name_starts_with=str(folder)) for page in data.by_page(): for item in page: - yield item + yield Path(item) def get_raw_file_stream(self, blob_name: Path) -> BytesIO: - blob_client = self.client.get_blob_client(str(blob_name)) + blob_client = self._client.get_blob_client(str(blob_name)) data = blob_client.download_blob().readall() return BytesIO(data) diff --git a/quixstreams/sources/community/file/origins/base.py b/quixstreams/sources/community/file/origins/base.py index 84974d531..b3ef10aab 100644 --- a/quixstreams/sources/community/file/origins/base.py +++ b/quixstreams/sources/community/file/origins/base.py @@ -1,34 +1,21 @@ -from abc import abstractmethod +from abc import ABC, abstractmethod from dataclasses import dataclass from pathlib import Path from typing import Any, BinaryIO, Iterable, Union -__all__ = ("FileOrigin",) +__all__ = ( + "Origin", + "ExternalOrigin", +) -@dataclass -class FileOrigin: +class Origin(ABC): """ An interface for interacting with a file-based client. Provides methods for navigating folders and retrieving/opening raw files. """ - _client: Any - _credentials: Union[dict, str] - root_location: Union[str, Path] - - @property - @abstractmethod - def client(self): ... - - """ - Set _client here. - - Circumvents pickling issues with multiprocessing by init-ing the client in a - later step (when Application.run() is called). - """ - @abstractmethod def file_collector(self, filepath: Path) -> Iterable[Path]: ... @@ -51,3 +38,11 @@ def get_raw_file_stream(self, filepath: Path) -> BinaryIO: ... Result should be ready for deserialization (and/or decompression). """ + + +@dataclass +class ExternalOrigin(Origin, ABC): + """An interface for interacting with an external file-based client""" + + _client: Any + root_location: Union[str, Path] diff --git a/quixstreams/sources/community/file/origins/local.py b/quixstreams/sources/community/file/origins/local.py index 3ad09f746..db6f4208d 100644 --- a/quixstreams/sources/community/file/origins/local.py +++ b/quixstreams/sources/community/file/origins/local.py @@ -1,13 +1,13 @@ from io import BytesIO from pathlib import Path -from typing import Generator, Optional +from typing import Generator -from .base import FileOrigin +from .base import Origin -__all__ = ("LocalFileOrigin",) +__all__ = ("LocalOrigin",) -class LocalFileOrigin(FileOrigin): +class LocalOrigin(Origin): def __init__( self, ): @@ -19,9 +19,7 @@ def __init__( def client(self): return - def file_collector( - self, filepath: Optional[Path] = None - ) -> Generator[Path, None, None]: + def file_collector(self, filepath: Path) -> Generator[Path, None, None]: if filepath.is_dir(): for i in sorted(filepath.iterdir(), key=lambda x: x.name): yield from self.file_collector(i) @@ -32,5 +30,4 @@ def get_folder_count(self, folder: Path) -> int: return len([f for f in folder.iterdir()]) def get_raw_file_stream(self, filepath: Path) -> BytesIO: - with open(filepath, "rb") as f: - return BytesIO(f.read()) + return BytesIO(filepath.read_bytes()) diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index 0a266437b..029ffead1 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -4,7 +4,7 @@ from pathlib import Path from typing import Generator, Optional, Union -from .base import FileOrigin +from .base import ExternalOrigin try: from boto3 import client as boto_client @@ -12,15 +12,15 @@ except ImportError as exc: raise ImportError( f"Package {exc.name} is missing: " - 'run "pip install quixstreams[aws]" to use S3FileOrigin' + 'run "pip install quixstreams[s3]" to use S3Origin' ) from exc logger = logging.getLogger(__name__) -__all__ = ("S3FileOrigin",) +__all__ = ("S3Origin",) -class S3FileOrigin(FileOrigin): +class S3Origin(ExternalOrigin): def __init__( self, aws_s3_bucket: str, @@ -32,7 +32,7 @@ def __init__( """ Configure IcebergSink to work with AWS Glue. - :param aws_s3_bucket: The S3 URI with bucket name only (ex: 's3://your-bucket'). + :param aws_s3_bucket: The S3 bucket name only (ex: 'your-bucket'). :param aws_region: The AWS region. NOTE: can alternatively set the AWS_REGION environment variable :param aws_access_key_id: the AWS access key ID. @@ -44,35 +44,35 @@ def __init__( NOTE: can alternatively set the AWS_ENDPOINT_URL_S3 environment variable """ self.root_location = aws_s3_bucket - self._client: Optional[S3Client] = None self._credentials = { "region_name": aws_region, "aws_access_key_id": aws_access_key_id, "aws_secret_access_key": aws_secret_access_key, "endpoint_url": aws_endpoint_url, } + # S3 client runs into pickling errors with multiprocessing. We can't set it + # until multiprocessing starts it. + # We can work around it by setting it during file collection + self._client: Optional[S3Client] = None - @property - def client(self): - if not self._client: - self._client: S3Client = boto_client("s3", **self._credentials) - return self._client + def _get_client(self) -> S3Client: + return boto_client("s3", **self._credentials) def get_raw_file_stream(self, filepath: Path) -> BytesIO: - data = self.client.get_object(Bucket=self.root_location, Key=str(filepath))[ + data = self._client.get_object(Bucket=self.root_location, Key=str(filepath))[ "Body" ].read() return BytesIO(data) def get_folder_count(self, folder: Path) -> int: - resp = self.client.list_objects( + resp = self._get_client().list_objects( Bucket=self.root_location, Prefix=str(folder), Delimiter="/" ) - self._client = None return len(resp["CommonPrefixes"]) def file_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None]: - resp = self.client.list_objects( + self._client = self._get_client() + resp = self._client.list_objects( Bucket=self.root_location, Prefix=str(folder), Delimiter="/", From 8a699aca232b960d280066bcc50d6c5f7e370a33 Mon Sep 17 00:00:00 2001 From: Tim Sawicki <136370015+tim-quix@users.noreply.github.com> Date: Wed, 27 Nov 2024 12:29:15 -0500 Subject: [PATCH 13/24] Update docs/connectors/sources/aws-s3-file-source.md Co-authored-by: Remy Gwaramadze --- docs/connectors/sources/aws-s3-file-source.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/aws-s3-file-source.md index c9e782393..76cea11eb 100644 --- a/docs/connectors/sources/aws-s3-file-source.md +++ b/docs/connectors/sources/aws-s3-file-source.md @@ -18,7 +18,7 @@ producing is matched as close as possible to the original. (per topic partition Install Quix Streams with the following optional dependencies: ```bash -pip install quixstreams[aws] +pip install quixstreams[s3] ``` ## How It Works From c26a2172052a62994fc8db5f237f5971ce0e8084 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 12:33:50 -0500 Subject: [PATCH 14/24] change bucket variable name --- docs/connectors/sources/aws-s3-file-source.md | 4 ++-- quixstreams/sources/community/file/file.py | 2 +- quixstreams/sources/community/file/origins/s3.py | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/aws-s3-file-source.md index 76cea11eb..cc2634448 100644 --- a/docs/connectors/sources/aws-s3-file-source.md +++ b/docs/connectors/sources/aws-s3-file-source.md @@ -53,7 +53,7 @@ from quixstreams.sources.community.file.origins import S3FileOrigin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") file_origin = S3FileOrigin( - aws_s3_bucket="", + bucket="", aws_access_key_id="", aws_secret_access_key="", aws_region="", @@ -79,7 +79,7 @@ Here are some important configurations to be aware of (see [File Source API](../ `S3FileOrigin`: -- `aws_s3_bucket`: The S3 bucket name only (ex: `"your-bucket"`). +- `bucket`: The S3 bucket name only (ex: `"your-bucket"`). - `aws_region`: AWS region (ex: us-east-1). **Note**: can alternatively set the `AWS_REGION` environment variable. - `aws_access_key_id`: AWS User key ID. diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index 0cf79e303..df75c8ff7 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -57,7 +57,7 @@ class FileSource(Source): app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") file_origin = S3Origin( - aws_s3_bucket="", + bucket="", aws_access_key_id="", aws_secret_access_key="", aws_region="", diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index 029ffead1..6995e44dc 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -23,7 +23,7 @@ class S3Origin(ExternalOrigin): def __init__( self, - aws_s3_bucket: str, + bucket: str, aws_region: Optional[str] = getenv("AWS_REGION"), aws_access_key_id: Optional[str] = getenv("AWS_ACCESS_KEY_ID"), aws_secret_access_key: Optional[str] = getenv("AWS_SECRET_ACCESS_KEY"), @@ -32,7 +32,7 @@ def __init__( """ Configure IcebergSink to work with AWS Glue. - :param aws_s3_bucket: The S3 bucket name only (ex: 'your-bucket'). + :param bucket: The S3 bucket name only (ex: 'your-bucket'). :param aws_region: The AWS region. NOTE: can alternatively set the AWS_REGION environment variable :param aws_access_key_id: the AWS access key ID. @@ -43,7 +43,7 @@ def __init__( to a locally hosted Kinesis. NOTE: can alternatively set the AWS_ENDPOINT_URL_S3 environment variable """ - self.root_location = aws_s3_bucket + self.root_location = bucket self._credentials = { "region_name": aws_region, "aws_access_key_id": aws_access_key_id, From bb12e078a5ce86e33b473c4398cd9367b57bf208 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 16:51:09 -0500 Subject: [PATCH 15/24] update local file docs --- docs/connectors/sources/local-file-source.md | 126 ++++++++++++++---- ...ws-s3-file-source.md => s3-file-source.md} | 19 ++- 2 files changed, 121 insertions(+), 24 deletions(-) rename docs/connectors/sources/{aws-s3-file-source.md => s3-file-source.md} (91%) diff --git a/docs/connectors/sources/local-file-source.md b/docs/connectors/sources/local-file-source.md index f538f9da1..d7ec77a18 100644 --- a/docs/connectors/sources/local-file-source.md +++ b/docs/connectors/sources/local-file-source.md @@ -1,4 +1,4 @@ -# File Source +# Local File Source !!! info @@ -6,24 +6,43 @@ To learn more about differences between Core and Community connectors, see the [Community and Core Connectors](../community-and-core.md) page. -This source enables reading from a localized file source, such as a JSONlines or Parquet -file. It also supports file (de)compression. +This source reads records from files at a local filepath and produces +them as messages to a kafka topic using any desired `StreamingDataFrame`-based transformations. The resulting messages can be produced in "replay" mode, where the time between record producing is matched as close as possible to the original. (per topic partition only). -The File Source connector is generally intended to be used alongside the related -[File Sink](../sinks/file-sink.md) (in terms of expected file and data formatting). + +## How To Install + +Simply install Quix Streams, no options required: + +```bash +pip install quixstreams +``` + +## How It Works + +`FileSource` steps through each folder within the provided path and dumps each record +contained in each file as a message to a Kafka topic. Folders are navigated in +lexicographical order. + +Records are read in a streaming fashion and committed after every file, offering +[at-least-once guarantees](#processingdelivery-guarantees). + +It can handle one given file type (ex. JSONlines or Parquet) at a time, and also +supports file decompression. + +You can learn more details about the [expected kafka message format](#message-data-formatschema) below. ## How To Use -To use a File Source, you need to create an instance of `FileSource` -and pass it to the `app.dataframe()` method. +Local File Source is the default configuration of the `FileSource` connector. -One important thing to note is that you should in general point to a single topic folder -(rather than a root folder with many topics) otherwise topic partitions may not line up correctly. +Simply hand the configured `FileSource` (without a `file_origin`) to your `SDF` +(`app.dataframe(source=)`). -For the full description of expected parameters, see the [File Source API](../../api-reference/sources.md#filesource) page. +For more details around various settings, see [configuration](#configuration). ```python from quixstreams import Application @@ -34,14 +53,36 @@ source = FileSource( filepath="/path/to/my/topic_folder", file_format="json", file_compression="gzip", - as_replay=True + as_replay=True, ) sdf = app.dataframe(source=source).print(metadata=True) +# YOUR LOGIC HERE! if __name__ == "__main__": app.run() ``` +## Configuration + +Here are some important configurations to be aware of (see [File Source API](../../api-reference/sources.md#filesource) for all parameters). + +### Required: + +- `filepath`: a filepath to recursively read through (exclude bucket name). + **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). + +### Optional: + +- `file_format`: what format the message files are in (ex: `"json"`, `"parquet"`). + **Advanced**: can optionally provide a `Format` instance (`file_compression` will then be ignored). + **Default**: `"json"` +- `file_compression`: what compression is used on the given files, if any (ex: `"gzip"`) + **Default**: `None` +- `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. + **Note**: Time delay will only be accurate _per partition_, NOT overall. + **Default**: `True` + + ## File hierarchy/structure The File Source expects a folder structure like so: @@ -61,29 +102,68 @@ The File Source expects a folder structure like so: This is the default structure generated by the File Sink. -## File data format/schema +## Message Data Format/Schema + +The expected file schema largely depends on the chosen +file format. -The expected data schema is largely dependent on the file format chosen. +For easiest use (especially alongside [`FileSink`](../sinks/file-sink.md)), +you can follow these patterns: -For easiest use with the [File Sink](../sinks/file-sink.md), you can follow these patterns: +### Row-based Formats (ex: JSON) + +Files should have records with the following fields, with `_value` being a +JSON-deserializable item: -- for row-based formats (like JSON), the expected data should have records -with the following fields, where value is the entirety of the message value, -ideally as a JSON-deserializable item: - `_key` - `_value` - `_timestamp` -- for columnar formats (like Parquet), they do not expect an explicit `value` -field; instead all columns should be included individually while including `_key` and `_timestamp`: + +This will result in the following Kafka message format for `Application`: + +- Message `key` will be the record `_key` as `bytes`. +- Message `value` will be the record `_value` as a `json`/`dict` +- Message `timestamp` will be the record `_timestamp` (ms). + +### Columnar Formats (ex: Parquet) +These do not expect an explicit `value` field; instead all columns should be included +individually while including `_key` and `_timestamp`: + - `_key` - `_timestamp` - `field_a` - - `field_b`... + - `field_b` + etc... + + +This will result in the following Kafka message format for `Application`: + +- Message `key` will be the record `_key` as `bytes`. +- Message `value` will be every record field except `_key` and `_timestamp` packed as a `json`/`dict` +- Message `timestamp` will be the record `_timestamp` (ms). + + +### Custom Schemas (Advanced) + +If the original files are not formatted as expected, custom loaders can be configured +on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(file_format=)`. + +Formats can be imported from `quixstreams.sources.community.file.formats`. + +## Processing/Delivery Guarantees + +This Source offers "at-least-once" guarantees with message delivery: messages are +guaranteed to be committed when a file is finished processing. + +However, it does not save any state/position: an unhandled exception will cause the +`Application` to fail, and rerunning the `Application` will begin processing from the +beginning (reproducing all previously processed messages). -etc... - ## Topic The default topic will have a partition count that reflects the partition count found -within the provided topic's folder structure. \ No newline at end of file +within the provided topic's folder structure. + +The default topic name the Application dumps to is based on the last folder name of +the `FileSource` `filepath` as: `source__`. diff --git a/docs/connectors/sources/aws-s3-file-source.md b/docs/connectors/sources/s3-file-source.md similarity index 91% rename from docs/connectors/sources/aws-s3-file-source.md rename to docs/connectors/sources/s3-file-source.md index cc2634448..34ec64bf4 100644 --- a/docs/connectors/sources/aws-s3-file-source.md +++ b/docs/connectors/sources/s3-file-source.md @@ -41,7 +41,7 @@ S3 File Source is just a special configuration of the `FileSource` connector. Simply provide it an `S3FileOrigin` (`FileSource(origin=)`). -Then, hand the configured `FileSource` to your SDF (`app.dataframe(source=)`). +Then, hand the configured `FileSource` to your `SDF` (`app.dataframe(source=)`). For more details around various settings, see [configuration](#configuration). @@ -108,6 +108,23 @@ Here are some important configurations to be aware of (see [File Source API](../ **Note**: Time delay will only be accurate _per partition_, NOT overall. **Default**: `True` +## File hierarchy/structure + +The File Source expects a folder structure like so: + +``` + my_sinked_topics/ + ├── topic_a/ # topic name (use this path to File Source!) + │ ├── 0/ # topic partition number + │ │ ├── 0000.ext # formatted offset files (ex: JSON) + │ │ └── 0011.ext + │ └── 1/ + │ ├── 0003.ext + │ └── 0016.ext + └── topic_b/ + └── etc... +``` + ## Message Data Format/Schema The expected file schema largely depends on the chosen From 07af8003bf8687924235ea6c2fe792a800db6198 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 17:13:11 -0500 Subject: [PATCH 16/24] add azure docs --- docs/connectors/sources/azure-file-source.md | 185 ++++++++++++++++++ docs/connectors/sources/s3-file-source.md | 10 +- .../sources/community/file/origins/azure.py | 4 + 3 files changed, 194 insertions(+), 5 deletions(-) create mode 100644 docs/connectors/sources/azure-file-source.md diff --git a/docs/connectors/sources/azure-file-source.md b/docs/connectors/sources/azure-file-source.md new file mode 100644 index 000000000..b3bcb538b --- /dev/null +++ b/docs/connectors/sources/azure-file-source.md @@ -0,0 +1,185 @@ +# Microsoft Azure File Source + +!!! info + + This is a **Community** connector. Test it before using in production. + + To learn more about differences between Core and Community connectors, see the [Community and Core Connectors](../community-and-core.md) page. + +This source reads records from files located in an Azure container path and produces +them as messages to a kafka topic using any desired `StreamingDataFrame`-based transformations. + +The resulting messages can be produced in "replay" mode, where the time between record +producing is matched as close as possible to the original. (per topic partition only). + + +## How To Install + +Install Quix Streams with the following optional dependencies: + +```bash +pip install quixstreams[azure] +``` + +## How It Works + +`FileSource` steps through each folder within the provided path and dumps each record +contained in each file as a message to a Kafka topic. Folders are navigated in +lexicographical order. + +Records are read in a streaming fashion and committed after every file, offering +[at-least-once guarantees](#processingdelivery-guarantees). + +It can handle one given file type (ex. JSONlines or Parquet) at a time, and also +supports file decompression. + +You can learn more details about the [expected kafka message format](#message-data-formatschema) below. + +## How To Use + +Azure File Source is just a special configuration of the `FileSource` connector. + +Simply provide it an `AzureOrigin` (`FileSource(origin=)`). + +Then, hand the configured `FileSource` to your `SDF` (`app.dataframe(source=)`). + +For more details around various settings, see [configuration](#configuration). + +```python +from quixstreams import Application +from quixstreams.sources.community.file import FileSource +from quixstreams.sources.community.file.origins import AzureOrigin + +app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") + +file_origin = AzureOrigin( + container="", + connection_string="", +) +source = FileSource( + filepath="path/to/your/topic_folder/", + file_origin=file_origin, + file_format="json", + file_compression="gzip", +) +sdf = app.dataframe(source=source).print(metadata=True) +# YOUR LOGIC HERE! + +if __name__ == "__main__": + app.run() +``` + +## Configuration + +Here are some important configurations to be aware of (see [File Source API](../../api-reference/sources.md#filesource) for all parameters). + +### Required: + +`AzureOrigin`: + +- `connection_string`: Azure client authentication string. +- `container`: Azure container name. + + +`FileSource`: + +- `filepath`: a filepath to recursively read through (exclude bucket name). + **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). +- `file_origin`: An `AzureOrigin` instance. + + +### Optional: + +`FileSource`: + +- `file_format`: what format the message files are in (ex: `"json"`, `"parquet"`). + **Advanced**: can optionally provide a `Format` instance (`file_compression` will then be ignored). + **Default**: `"json"` +- `file_compression`: what compression is used on the given files, if any (ex: `"gzip"`) + **Default**: `None` +- `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. + **Note**: Time delay will only be accurate _per partition_, NOT overall. + **Default**: `True` + +## File hierarchy/structure + +The File Source expects a folder structure like so: + +``` + my_sinked_topics/ + ├── topic_a/ # topic name (use this path to File Source!) + │ ├── 0/ # topic partition number + │ │ ├── 0000.ext # formatted offset files (ex: JSON) + │ │ └── 0011.ext + │ └── 1/ + │ ├── 0003.ext + │ └── 0016.ext + └── topic_b/ + └── etc... +``` + +## Message Data Format/Schema + +The expected file schema largely depends on the chosen +file format. + +For easiest use (especially alongside [`FileSink`](../sinks/file-sink.md)), +you can follow these patterns: + +### Row-based Formats (ex: JSON) + +Files should have records with the following fields, with `_value` being a +JSON-deserializable item: + + - `_key` + - `_value` + - `_timestamp` + + +This will result in the following Kafka message format for `Application`: + +- Message `key` will be the record `_key` as `bytes`. +- Message `value` will be the record `_value` as a `json`/`dict` +- Message `timestamp` will be the record `_timestamp` (ms). + +### Columnar Formats (ex: Parquet) +These do not expect an explicit `value` field; instead all columns should be included +individually while including `_key` and `_timestamp`: + + - `_key` + - `_timestamp` + - `field_a` + - `field_b` + etc... + + +This will result in the following Kafka message format for `Application`: + +- Message `key` will be the record `_key` as `bytes`. +- Message `value` will be every record field except `_key` and `_timestamp` packed as a `json`/`dict` +- Message `timestamp` will be the record `_timestamp` (ms). + + +### Custom Schemas (Advanced) + +If the original files are not formatted as expected, custom loaders can be configured +on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(file_format=)`. + +Formats can be imported from `quixstreams.sources.community.file.formats`. + +## Processing/Delivery Guarantees + +This Source offers "at-least-once" guarantees with message delivery: messages are +guaranteed to be committed when a file is finished processing. + +However, it does not save any state/position: an unhandled exception will cause the +`Application` to fail, and rerunning the `Application` will begin processing from the +beginning (reproducing all previously processed messages). + +## Topic + +The default topic will have a partition count that reflects the partition count found +within the provided topic's folder structure. + +The default topic name the Application dumps to is based on the last folder name of +the `FileSource` `filepath` as: `source__`. diff --git a/docs/connectors/sources/s3-file-source.md b/docs/connectors/sources/s3-file-source.md index 34ec64bf4..09ab62ac4 100644 --- a/docs/connectors/sources/s3-file-source.md +++ b/docs/connectors/sources/s3-file-source.md @@ -39,7 +39,7 @@ You can learn more details about the [expected kafka message format](#message-da S3 File Source is just a special configuration of the `FileSource` connector. -Simply provide it an `S3FileOrigin` (`FileSource(origin=)`). +Simply provide it an `S3Origin` (`FileSource(origin=)`). Then, hand the configured `FileSource` to your `SDF` (`app.dataframe(source=)`). @@ -48,11 +48,11 @@ For more details around various settings, see [configuration](#configuration). ```python from quixstreams import Application from quixstreams.sources.community.file import FileSource -from quixstreams.sources.community.file.origins import S3FileOrigin +from quixstreams.sources.community.file.origins import S3Origin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") -file_origin = S3FileOrigin( +file_origin = S3Origin( bucket="", aws_access_key_id="", aws_secret_access_key="", @@ -77,7 +77,7 @@ Here are some important configurations to be aware of (see [File Source API](../ ### Required: -`S3FileOrigin`: +`S3Origin`: - `bucket`: The S3 bucket name only (ex: `"your-bucket"`). - `aws_region`: AWS region (ex: us-east-1). @@ -92,7 +92,7 @@ Here are some important configurations to be aware of (see [File Source API](../ - `filepath`: a filepath to recursively read through (exclude bucket name). **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). -- `file_origin`: An `S3FileOrigin` instance. +- `file_origin`: An `S3Origin` instance. ### Optional: diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py index 2e99143cc..133192adf 100644 --- a/quixstreams/sources/community/file/origins/azure.py +++ b/quixstreams/sources/community/file/origins/azure.py @@ -27,6 +27,10 @@ def __init__( connection_string: str, container: str, ): + """ + :param connection_string: Azure client authentication string. + :param container: Azure container name. + """ self._client: Optional[ContainerClient] = self._get_client(connection_string) self.root_location = container From 72cba515db5789bf000c976f5267c3ab3b8d715e Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Wed, 27 Nov 2024 22:59:03 -0500 Subject: [PATCH 17/24] clean up azure stuff and other small tweaks --- docs/connectors/sources/azure-file-source.md | 20 +++++++++++++ docs/connectors/sources/s3-file-source.md | 22 +++++++++++++++ quixstreams/sources/community/file/file.py | 2 +- .../sources/community/file/origins/azure.py | 28 +++++++++++-------- .../sources/community/file/origins/s3.py | 8 +++--- 5 files changed, 64 insertions(+), 16 deletions(-) diff --git a/docs/connectors/sources/azure-file-source.md b/docs/connectors/sources/azure-file-source.md index b3bcb538b..80569cc21 100644 --- a/docs/connectors/sources/azure-file-source.md +++ b/docs/connectors/sources/azure-file-source.md @@ -183,3 +183,23 @@ within the provided topic's folder structure. The default topic name the Application dumps to is based on the last folder name of the `FileSource` `filepath` as: `source__`. + + +## Testing Locally + +Rather than connect to Azure, you can alternatively test your application using a local +emulated Azure host via Docker: + +1. Execute in terminal: + + ```bash + docker run --rm -d --name azurite \ + -p 10000:10000 \ + mcr.microsoft.com/azure-storage/azurite:latest + ``` + +2. Set `connection_string` for `AzureOrigin` to: + +```python +"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" +``` diff --git a/docs/connectors/sources/s3-file-source.md b/docs/connectors/sources/s3-file-source.md index 09ab62ac4..8674a6159 100644 --- a/docs/connectors/sources/s3-file-source.md +++ b/docs/connectors/sources/s3-file-source.md @@ -190,3 +190,25 @@ within the provided topic's folder structure. The default topic name the Application dumps to is based on the last folder name of the `FileSource` `filepath` as: `source__`. + +## Testing Locally + +Rather than connect to AWS, you can alternatively test your application using a local +emulated S3 host via Docker: + +1. Execute in terminal: + + ```bash + docker run --rm -d --name s3 \ + -p 4566:4566 \ + -e SERVICES=s3 \ + -e EDGE_PORT=4566 \ + -e DEBUG=1 \ + localstack/localstack:latest + ``` + +2. Set `aws_endpoint_url` for `S3Origin` _OR_ the `AWS_ENDPOINT_URL_S3` + environment variable to `http://localhost:4566` + +3. Set all other `aws_` parameters for `S3Origin` to _any_ string. +They will not be used, but they must still be populated! diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index df75c8ff7..f1e324313 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -152,7 +152,7 @@ def default_topic(self) -> Topic: """ topic = super().default_topic() topic.config = TopicConfig( - num_partitions=self._origin.get_folder_count(self._filepath), + num_partitions=self._origin.get_folder_count(self._filepath) or 1, replication_factor=1, ) return topic diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py index 133192adf..4597e9346 100644 --- a/quixstreams/sources/community/file/origins/azure.py +++ b/quixstreams/sources/community/file/origins/azure.py @@ -1,6 +1,7 @@ +import os from io import BytesIO from pathlib import Path -from typing import Generator, Optional +from typing import Generator from .base import ExternalOrigin @@ -17,9 +18,17 @@ class AzureOrigin(ExternalOrigin): - def get_folder_count(self, filepath: Path) -> int: - data = self._client.list_blobs(name_starts_with=str(filepath)) - folders = [f for page in data.by_page() for f in page.get("blob_prefixes", [])] + def get_folder_count(self, path: Path) -> int: + """ + This is a simplified version of the recommended way to retrieve folder + names based on the azure SDK docs examples. + """ + path = f"{path}/" + folders = set() + for blob in self._client.list_blobs(name_starts_with=path): + relative_dir = os.path.dirname(os.path.relpath(blob.name, path)) + if relative_dir and ("/" not in relative_dir): + folders.add(relative_dir) return len(folders) def __init__( @@ -31,15 +40,12 @@ def __init__( :param connection_string: Azure client authentication string. :param container: Azure container name. """ - self._client: Optional[ContainerClient] = self._get_client(connection_string) self.root_location = container + self._client = self._get_client(connection_string) - def _get_client(self, auth: str): - if not self._client: - blob_client = BlobServiceClient.from_connection_string(auth) - container_client = blob_client.get_container_client(self.root_location) - self._client: ContainerClient = container_client - return self._client + def _get_client(self, auth: str) -> ContainerClient: + blob_client = BlobServiceClient.from_connection_string(auth) + return blob_client.get_container_client(self.root_location) def file_collector(self, folder: Path) -> Generator[Path, None, None]: data = self._client.list_blob_names(name_starts_with=str(folder)) diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index 6995e44dc..ae4c9ecb6 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -40,7 +40,7 @@ def __init__( :param aws_secret_access_key: the AWS secret access key. NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable :param aws_endpoint_url: the endpoint URL to use; only required for connecting - to a locally hosted Kinesis. + to a locally hosted S3. NOTE: can alternatively set the AWS_ENDPOINT_URL_S3 environment variable """ self.root_location = bucket @@ -66,7 +66,7 @@ def get_raw_file_stream(self, filepath: Path) -> BytesIO: def get_folder_count(self, folder: Path) -> int: resp = self._get_client().list_objects( - Bucket=self.root_location, Prefix=str(folder), Delimiter="/" + Bucket=self.root_location, Prefix=f"{folder}/", Delimiter="/" ) return len(resp["CommonPrefixes"]) @@ -77,8 +77,8 @@ def file_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None Prefix=str(folder), Delimiter="/", ) - for folder in resp.get("CommonPrefixes", []): - yield from self.file_collector(folder["Prefix"]) + for _folder in resp.get("CommonPrefixes", []): + yield from self.file_collector(_folder["Prefix"]) for file in resp.get("Contents", []): yield Path(file["Key"]) From f581e287e99d7d0a21e57185714b2b0946823f12 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 13:35:49 -0500 Subject: [PATCH 18/24] clean up after second round of code review --- docs/connectors/sources/azure-file-source.md | 24 +++++----- docs/connectors/sources/local-file-source.md | 22 +++++----- docs/connectors/sources/s3-file-source.md | 24 +++++----- quixstreams/sources/community/file/file.py | 44 +++++++++---------- .../sources/community/file/origins/azure.py | 40 ++++++++--------- .../sources/community/file/origins/base.py | 20 ++------- .../sources/community/file/origins/local.py | 15 +------ .../sources/community/file/origins/s3.py | 32 +++++++------- 8 files changed, 99 insertions(+), 122 deletions(-) diff --git a/docs/connectors/sources/azure-file-source.md b/docs/connectors/sources/azure-file-source.md index 80569cc21..08fc4c7c7 100644 --- a/docs/connectors/sources/azure-file-source.md +++ b/docs/connectors/sources/azure-file-source.md @@ -52,15 +52,15 @@ from quixstreams.sources.community.file.origins import AzureOrigin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") -file_origin = AzureOrigin( +origin = AzureOrigin( container="", connection_string="", ) source = FileSource( - filepath="path/to/your/topic_folder/", - file_origin=file_origin, - file_format="json", - file_compression="gzip", + directory="path/to/your/topic_folder/", + origin=origin, + format="json", + compression="gzip", ) sdf = app.dataframe(source=source).print(metadata=True) # YOUR LOGIC HERE! @@ -83,19 +83,19 @@ Here are some important configurations to be aware of (see [File Source API](../ `FileSource`: -- `filepath`: a filepath to recursively read through (exclude bucket name). +- `directory`: a directory to recursively read through (exclude container name). **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). -- `file_origin`: An `AzureOrigin` instance. +- `origin`: An `AzureOrigin` instance. ### Optional: `FileSource`: -- `file_format`: what format the message files are in (ex: `"json"`, `"parquet"`). - **Advanced**: can optionally provide a `Format` instance (`file_compression` will then be ignored). +- `format`: what format the message files are in (ex: `"json"`, `"parquet"`). + **Advanced**: can optionally provide a `Format` instance (`compression` will then be ignored). **Default**: `"json"` -- `file_compression`: what compression is used on the given files, if any (ex: `"gzip"`) +- `compression`: what compression is used on the given files, if any (ex: `"gzip"`) **Default**: `None` - `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. **Note**: Time delay will only be accurate _per partition_, NOT overall. @@ -163,7 +163,7 @@ This will result in the following Kafka message format for `Application`: ### Custom Schemas (Advanced) If the original files are not formatted as expected, custom loaders can be configured -on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(file_format=)`. +on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(format=)`. Formats can be imported from `quixstreams.sources.community.file.formats`. @@ -182,7 +182,7 @@ The default topic will have a partition count that reflects the partition count within the provided topic's folder structure. The default topic name the Application dumps to is based on the last folder name of -the `FileSource` `filepath` as: `source__`. +the `FileSource` `directory` as: `source__`. ## Testing Locally diff --git a/docs/connectors/sources/local-file-source.md b/docs/connectors/sources/local-file-source.md index d7ec77a18..89a48d7d3 100644 --- a/docs/connectors/sources/local-file-source.md +++ b/docs/connectors/sources/local-file-source.md @@ -6,7 +6,7 @@ To learn more about differences between Core and Community connectors, see the [Community and Core Connectors](../community-and-core.md) page. -This source reads records from files at a local filepath and produces +This source reads records from files at a local directory and produces them as messages to a kafka topic using any desired `StreamingDataFrame`-based transformations. The resulting messages can be produced in "replay" mode, where the time between record @@ -39,7 +39,7 @@ You can learn more details about the [expected kafka message format](#message-da Local File Source is the default configuration of the `FileSource` connector. -Simply hand the configured `FileSource` (without a `file_origin`) to your `SDF` +Simply hand the configured `FileSource` (without a `origin`) to your `SDF` (`app.dataframe(source=)`). For more details around various settings, see [configuration](#configuration). @@ -50,9 +50,9 @@ from quixstreams.sources.community.file import FileSource app = Application(broker_address="localhost:9092") source = FileSource( - filepath="/path/to/my/topic_folder", - file_format="json", - file_compression="gzip", + directory="/path/to/my/topic_folder", + format="json", + compression="gzip", as_replay=True, ) sdf = app.dataframe(source=source).print(metadata=True) @@ -68,15 +68,15 @@ Here are some important configurations to be aware of (see [File Source API](../ ### Required: -- `filepath`: a filepath to recursively read through (exclude bucket name). +- `directory`: a directory to recursively read through (exclude bucket name). **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). ### Optional: -- `file_format`: what format the message files are in (ex: `"json"`, `"parquet"`). - **Advanced**: can optionally provide a `Format` instance (`file_compression` will then be ignored). +- `format`: what format the message files are in (ex: `"json"`, `"parquet"`). + **Advanced**: can optionally provide a `Format` instance (`compression` will then be ignored). **Default**: `"json"` -- `file_compression`: what compression is used on the given files, if any (ex: `"gzip"`) +- `compression`: what compression is used on the given files, if any (ex: `"gzip"`) **Default**: `None` - `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. **Note**: Time delay will only be accurate _per partition_, NOT overall. @@ -147,7 +147,7 @@ This will result in the following Kafka message format for `Application`: ### Custom Schemas (Advanced) If the original files are not formatted as expected, custom loaders can be configured -on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(file_format=)`. +on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(format=)`. Formats can be imported from `quixstreams.sources.community.file.formats`. @@ -166,4 +166,4 @@ The default topic will have a partition count that reflects the partition count within the provided topic's folder structure. The default topic name the Application dumps to is based on the last folder name of -the `FileSource` `filepath` as: `source__`. +the `FileSource` `directory` as: `source__`. diff --git a/docs/connectors/sources/s3-file-source.md b/docs/connectors/sources/s3-file-source.md index 8674a6159..acb75d6da 100644 --- a/docs/connectors/sources/s3-file-source.md +++ b/docs/connectors/sources/s3-file-source.md @@ -52,17 +52,17 @@ from quixstreams.sources.community.file.origins import S3Origin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") -file_origin = S3Origin( +origin = S3Origin( bucket="", aws_access_key_id="", aws_secret_access_key="", aws_region="", ) source = FileSource( - filepath="path/to/your/topic_folder/", - file_origin=file_origin, - file_format="json", - file_compression="gzip", + directory="path/to/your/topic_folder/", + origin=origin, + format="json", + compression="gzip", ) sdf = app.dataframe(source=source).print(metadata=True) # YOUR LOGIC HERE! @@ -90,19 +90,19 @@ Here are some important configurations to be aware of (see [File Source API](../ `FileSource`: -- `filepath`: a filepath to recursively read through (exclude bucket name). +- `directory`: a directory to recursively read through (exclude bucket name). **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). -- `file_origin`: An `S3Origin` instance. +- `origin`: An `S3Origin` instance. ### Optional: `FileSource`: -- `file_format`: what format the message files are in (ex: `"json"`, `"parquet"`). - **Advanced**: can optionally provide a `Format` instance (`file_compression` will then be ignored). +- `format`: what format the message files are in (ex: `"json"`, `"parquet"`). + **Advanced**: can optionally provide a `Format` instance (`compression` will then be ignored). **Default**: `"json"` -- `file_compression`: what compression is used on the given files, if any (ex: `"gzip"`) +- `compression`: what compression is used on the given files, if any (ex: `"gzip"`) **Default**: `None` - `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. **Note**: Time delay will only be accurate _per partition_, NOT overall. @@ -170,7 +170,7 @@ This will result in the following Kafka message format for `Application`: ### Custom Schemas (Advanced) If the original files are not formatted as expected, custom loaders can be configured -on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(file_format=)`. +on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(format=)`. Formats can be imported from `quixstreams.sources.community.file.formats`. @@ -189,7 +189,7 @@ The default topic will have a partition count that reflects the partition count within the provided topic's folder structure. The default topic name the Application dumps to is based on the last folder name of -the `FileSource` `filepath` as: `source__`. +the `FileSource` `directory` as: `source__`. ## Testing Locally diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index f1e324313..f53772ebf 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -56,17 +56,17 @@ class FileSource(Source): app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") - file_origin = S3Origin( + origin = S3Origin( bucket="", aws_access_key_id="", aws_secret_access_key="", aws_region="", ) source = FileSource( - filepath="path/to/your/topic_folder/", - file_origin=file_origin, - file_format="json", - file_compression="gzip", + directory="path/to/your/topic_folder/", + origin=origin, + format="json", + compression="gzip", ) sdf = app.dataframe(source=source).print(metadata=True) # YOUR LOGIC HERE! @@ -78,22 +78,22 @@ class FileSource(Source): def __init__( self, - filepath: Union[str, Path], - file_format: Union[Format, FormatName] = "json", - file_origin: Origin = LocalOrigin(), - file_compression: Optional[CompressionName] = None, + directory: Union[str, Path], + format: Union[Format, FormatName] = "json", + origin: Origin = LocalOrigin(), + compression: Optional[CompressionName] = None, as_replay: bool = True, name: Optional[str] = None, shutdown_timeout: float = 10, ): """ - :param filepath: a filepath to recursively read through; it is recommended to + :param directory: a directory to recursively read through; it is recommended to provide the path to a given topic folder (ex: `/path/to/topic_a`). - :param file_format: what format the message files are in (ex: json, parquet). - Optionally, can provide a `Format` instance if more than file_compression - is necessary to define (file_compression will then be ignored). - :param file_origin: an Origin type (defaults to reading local files). - :param file_compression: what compression is used on the given files, if any. + :param format: what format the message files are in (ex: json, parquet). + Optionally, can provide a `Format` instance if more than compression + is necessary to define (compression will then be ignored). + :param origin: an Origin type (defaults to reading local files). + :param compression: what compression is used on the given files, if any. :param as_replay: Produce the messages with the original time delay between them. Otherwise, produce the messages as fast as possible. NOTE: Time delay will only be accurate per partition, NOT overall. @@ -101,14 +101,14 @@ def __init__( :param shutdown_timeout: Time in seconds the application waits for the source to gracefully shutdown """ - self._filepath = Path(filepath) - self._origin = file_origin - self._formatter = _get_formatter(file_format, file_compression) + self._directory = Path(directory) + self._origin = origin + self._formatter = _get_formatter(format, compression) self._as_replay = as_replay self._previous_timestamp = None self._previous_partition = None super().__init__( - name=name or self._filepath.name, shutdown_timeout=shutdown_timeout + name=name or self._directory.name, shutdown_timeout=shutdown_timeout ) def _replay_delay(self, current_timestamp: int): @@ -152,15 +152,15 @@ def default_topic(self) -> Topic: """ topic = super().default_topic() topic.config = TopicConfig( - num_partitions=self._origin.get_folder_count(self._filepath) or 1, + num_partitions=self._origin.get_folder_count(self._directory) or 1, replication_factor=1, ) return topic def run(self): while self._running: - logger.info(f"Reading files from topic {self._filepath.name}") - for file in self._origin.file_collector(self._filepath): + logger.info(f"Reading files from topic {self._directory.name}") + for file in self._origin.file_collector(self._directory): logger.debug(f"Reading file {file}") self._check_file_partition_number(file) filestream = self._origin.get_raw_file_stream(file) diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py index 4597e9346..dbfb0c638 100644 --- a/quixstreams/sources/community/file/origins/azure.py +++ b/quixstreams/sources/community/file/origins/azure.py @@ -3,7 +3,7 @@ from pathlib import Path from typing import Generator -from .base import ExternalOrigin +from .base import Origin try: from azure.storage.blob import BlobServiceClient @@ -14,23 +14,10 @@ 'run "pip install quixstreams[azure]" to use AzureOrigin' ) from exc -__all__ = ("AzureOrigin",) +__all__ = ("AzureFilesOrigin",) -class AzureOrigin(ExternalOrigin): - def get_folder_count(self, path: Path) -> int: - """ - This is a simplified version of the recommended way to retrieve folder - names based on the azure SDK docs examples. - """ - path = f"{path}/" - folders = set() - for blob in self._client.list_blobs(name_starts_with=path): - relative_dir = os.path.dirname(os.path.relpath(blob.name, path)) - if relative_dir and ("/" not in relative_dir): - folders.add(relative_dir) - return len(folders) - +class AzureFilesOrigin(Origin): def __init__( self, connection_string: str, @@ -47,13 +34,26 @@ def _get_client(self, auth: str) -> ContainerClient: blob_client = BlobServiceClient.from_connection_string(auth) return blob_client.get_container_client(self.root_location) - def file_collector(self, folder: Path) -> Generator[Path, None, None]: - data = self._client.list_blob_names(name_starts_with=str(folder)) + def file_collector(self, filepath: Path) -> Generator[Path, None, None]: + data = self._client.list_blob_names(name_starts_with=str(filepath)) for page in data.by_page(): for item in page: yield Path(item) - def get_raw_file_stream(self, blob_name: Path) -> BytesIO: - blob_client = self._client.get_blob_client(str(blob_name)) + def get_folder_count(self, directory: Path) -> int: + """ + This is a simplified version of the recommended way to retrieve folder + names based on the azure SDK docs examples. + """ + path = f"{directory}/" + folders = set() + for blob in self._client.list_blobs(name_starts_with=path): + relative_dir = os.path.dirname(os.path.relpath(blob.name, path)) + if relative_dir and ("/" not in relative_dir): + folders.add(relative_dir) + return len(folders) + + def get_raw_file_stream(self, filepath: Path) -> BytesIO: + blob_client = self._client.get_blob_client(str(filepath)) data = blob_client.download_blob().readall() return BytesIO(data) diff --git a/quixstreams/sources/community/file/origins/base.py b/quixstreams/sources/community/file/origins/base.py index b3ef10aab..927ccef0d 100644 --- a/quixstreams/sources/community/file/origins/base.py +++ b/quixstreams/sources/community/file/origins/base.py @@ -1,12 +1,8 @@ from abc import ABC, abstractmethod -from dataclasses import dataclass from pathlib import Path -from typing import Any, BinaryIO, Iterable, Union +from typing import BinaryIO, Iterable -__all__ = ( - "Origin", - "ExternalOrigin", -) +__all__ = ("Origin",) class Origin(ABC): @@ -26,9 +22,9 @@ def file_collector(self, filepath: Path) -> Iterable[Path]: ... """ @abstractmethod - def get_folder_count(self, folder: Path) -> int: ... + def get_folder_count(self, directory: Path) -> int: ... - """Counts the number of folders at filepath to assume partition counts.""" + """Counts the number of folders at directory to assume partition counts.""" @abstractmethod def get_raw_file_stream(self, filepath: Path) -> BinaryIO: ... @@ -38,11 +34,3 @@ def get_raw_file_stream(self, filepath: Path) -> BinaryIO: ... Result should be ready for deserialization (and/or decompression). """ - - -@dataclass -class ExternalOrigin(Origin, ABC): - """An interface for interacting with an external file-based client""" - - _client: Any - root_location: Union[str, Path] diff --git a/quixstreams/sources/community/file/origins/local.py b/quixstreams/sources/community/file/origins/local.py index db6f4208d..5bfb1980c 100644 --- a/quixstreams/sources/community/file/origins/local.py +++ b/quixstreams/sources/community/file/origins/local.py @@ -8,17 +8,6 @@ class LocalOrigin(Origin): - def __init__( - self, - ): - self._client = None - self._credentials = {} - self.root_location = "/" - - @property - def client(self): - return - def file_collector(self, filepath: Path) -> Generator[Path, None, None]: if filepath.is_dir(): for i in sorted(filepath.iterdir(), key=lambda x: x.name): @@ -26,8 +15,8 @@ def file_collector(self, filepath: Path) -> Generator[Path, None, None]: else: yield filepath - def get_folder_count(self, folder: Path) -> int: - return len([f for f in folder.iterdir()]) + def get_folder_count(self, directory: Path) -> int: + return len([f for f in directory.iterdir()]) def get_raw_file_stream(self, filepath: Path) -> BytesIO: return BytesIO(filepath.read_bytes()) diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index ae4c9ecb6..b5aa2ef6b 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -4,7 +4,7 @@ from pathlib import Path from typing import Generator, Optional, Union -from .base import ExternalOrigin +from .base import Origin try: from boto3 import client as boto_client @@ -20,7 +20,7 @@ __all__ = ("S3Origin",) -class S3Origin(ExternalOrigin): +class S3Origin(Origin): def __init__( self, bucket: str, @@ -58,23 +58,11 @@ def __init__( def _get_client(self) -> S3Client: return boto_client("s3", **self._credentials) - def get_raw_file_stream(self, filepath: Path) -> BytesIO: - data = self._client.get_object(Bucket=self.root_location, Key=str(filepath))[ - "Body" - ].read() - return BytesIO(data) - - def get_folder_count(self, folder: Path) -> int: - resp = self._get_client().list_objects( - Bucket=self.root_location, Prefix=f"{folder}/", Delimiter="/" - ) - return len(resp["CommonPrefixes"]) - - def file_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None]: + def file_collector(self, filepath: Union[str, Path]) -> Generator[Path, None, None]: self._client = self._get_client() resp = self._client.list_objects( Bucket=self.root_location, - Prefix=str(folder), + Prefix=str(filepath), Delimiter="/", ) for _folder in resp.get("CommonPrefixes", []): @@ -82,3 +70,15 @@ def file_collector(self, folder: Union[str, Path]) -> Generator[Path, None, None for file in resp.get("Contents", []): yield Path(file["Key"]) + + def get_folder_count(self, directory: Path) -> int: + resp = self._get_client().list_objects( + Bucket=self.root_location, Prefix=f"{directory}/", Delimiter="/" + ) + return len(resp["CommonPrefixes"]) + + def get_raw_file_stream(self, filepath: Path) -> BytesIO: + data = self._client.get_object(Bucket=self.root_location, Key=str(filepath))[ + "Body" + ].read() + return BytesIO(data) From 47269288e1c6ba58c1a85b2b1b0f9f08ce0da501 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 13:52:51 -0500 Subject: [PATCH 19/24] doc inclusions and cleanup --- docs/build/build.py | 2 ++ docs/connectors/sources/azure-file-source.md | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/build/build.py b/docs/build/build.py index 102c626e4..0c62146b2 100644 --- a/docs/build/build.py +++ b/docs/build/build.py @@ -145,6 +145,8 @@ "quixstreams.sources.core.kafka.quix", "quixstreams.sources.community.file.file", "quixstreams.sources.community.file.compressions.gzip", + "quixstreams.sources.community.file.origins.local", + "quixstreams.sources.community.file.origins.s3", "quixstreams.sources.community.file.formats.json", "quixstreams.sources.community.file.formats.parquet", "quixstreams.sources.community.kinesis.kinesis", diff --git a/docs/connectors/sources/azure-file-source.md b/docs/connectors/sources/azure-file-source.md index 08fc4c7c7..b025e1064 100644 --- a/docs/connectors/sources/azure-file-source.md +++ b/docs/connectors/sources/azure-file-source.md @@ -48,11 +48,11 @@ For more details around various settings, see [configuration](#configuration). ```python from quixstreams import Application from quixstreams.sources.community.file import FileSource -from quixstreams.sources.community.file.origins import AzureOrigin +from quixstreams.sources.community.file.origins import AzureFilesOrigin app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") -origin = AzureOrigin( +origin = AzureFilesOrigin( container="", connection_string="", ) From 533115237a23a367c4926cd8f2a6f0a2f19b6574 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 14:59:44 -0500 Subject: [PATCH 20/24] some tweaks for consistency across source and sink --- docs/connectors/sources/s3-file-source.md | 8 ++++---- quixstreams/sources/community/file/origins/s3.py | 12 ++++++------ 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/connectors/sources/s3-file-source.md b/docs/connectors/sources/s3-file-source.md index acb75d6da..c9dafa159 100644 --- a/docs/connectors/sources/s3-file-source.md +++ b/docs/connectors/sources/s3-file-source.md @@ -56,7 +56,7 @@ origin = S3Origin( bucket="", aws_access_key_id="", aws_secret_access_key="", - aws_region="", + region_name="", ) source = FileSource( directory="path/to/your/topic_folder/", @@ -80,7 +80,7 @@ Here are some important configurations to be aware of (see [File Source API](../ `S3Origin`: - `bucket`: The S3 bucket name only (ex: `"your-bucket"`). -- `aws_region`: AWS region (ex: us-east-1). +- `region_name`: AWS region (ex: us-east-1). **Note**: can alternatively set the `AWS_REGION` environment variable. - `aws_access_key_id`: AWS User key ID. **Note**: can alternatively set the `AWS_ACCESS_KEY_ID` environment variable. @@ -90,7 +90,7 @@ Here are some important configurations to be aware of (see [File Source API](../ `FileSource`: -- `directory`: a directory to recursively read through (exclude bucket name). +- `directory`: a directory to recursively read through (exclude bucket name or starting "/"). **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). - `origin`: An `S3Origin` instance. @@ -207,7 +207,7 @@ emulated S3 host via Docker: localstack/localstack:latest ``` -2. Set `aws_endpoint_url` for `S3Origin` _OR_ the `AWS_ENDPOINT_URL_S3` +2. Set `endpoint_url` for `S3Origin` _OR_ the `AWS_ENDPOINT_URL_S3` environment variable to `http://localhost:4566` 3. Set all other `aws_` parameters for `S3Origin` to _any_ string. diff --git a/quixstreams/sources/community/file/origins/s3.py b/quixstreams/sources/community/file/origins/s3.py index b5aa2ef6b..ca30b5a75 100644 --- a/quixstreams/sources/community/file/origins/s3.py +++ b/quixstreams/sources/community/file/origins/s3.py @@ -24,31 +24,31 @@ class S3Origin(Origin): def __init__( self, bucket: str, - aws_region: Optional[str] = getenv("AWS_REGION"), + region_name: Optional[str] = getenv("AWS_REGION"), aws_access_key_id: Optional[str] = getenv("AWS_ACCESS_KEY_ID"), aws_secret_access_key: Optional[str] = getenv("AWS_SECRET_ACCESS_KEY"), - aws_endpoint_url: Optional[str] = getenv("AWS_ENDPOINT_URL_S3"), + endpoint_url: Optional[str] = getenv("AWS_ENDPOINT_URL_S3"), ): """ Configure IcebergSink to work with AWS Glue. :param bucket: The S3 bucket name only (ex: 'your-bucket'). - :param aws_region: The AWS region. + :param region_name: The AWS region. NOTE: can alternatively set the AWS_REGION environment variable :param aws_access_key_id: the AWS access key ID. NOTE: can alternatively set the AWS_ACCESS_KEY_ID environment variable :param aws_secret_access_key: the AWS secret access key. NOTE: can alternatively set the AWS_SECRET_ACCESS_KEY environment variable - :param aws_endpoint_url: the endpoint URL to use; only required for connecting + :param endpoint_url: the endpoint URL to use; only required for connecting to a locally hosted S3. NOTE: can alternatively set the AWS_ENDPOINT_URL_S3 environment variable """ self.root_location = bucket self._credentials = { - "region_name": aws_region, + "region_name": region_name, "aws_access_key_id": aws_access_key_id, "aws_secret_access_key": aws_secret_access_key, - "endpoint_url": aws_endpoint_url, + "endpoint_url": endpoint_url, } # S3 client runs into pickling errors with multiprocessing. We can't set it # until multiprocessing starts it. From bd1bdff6e5a447ba0cf2c26b5009e39e88c963e4 Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 15:51:03 -0500 Subject: [PATCH 21/24] adjust replay to only sleep with large enough deltas --- quixstreams/sources/community/file/file.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index f53772ebf..3f3eee4fe 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -117,10 +117,10 @@ def _replay_delay(self, current_timestamp: int): based on their timestamps. """ if self._previous_timestamp is not None: - time_diff = (current_timestamp - self._previous_timestamp) / 1000 - if time_diff > 0: - logger.debug(f"Sleeping for {time_diff} seconds...") - sleep(time_diff) + time_diff_seconds = (current_timestamp - self._previous_timestamp) / 1000 + if time_diff_seconds > 0.01: # only sleep when diff is "big enough" + logger.debug(f"Sleeping for {time_diff_seconds} seconds...") + sleep(time_diff_seconds) self._previous_timestamp = current_timestamp def _check_file_partition_number(self, file: Path): From 2c8483b8f4fde6e146b8c28eef47d38f4c45449e Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 22:11:57 -0500 Subject: [PATCH 22/24] adjust replay speed to accept floats rather than a bool --- docs/connectors/sources/azure-file-source.md | 6 ++++-- docs/connectors/sources/local-file-source.md | 6 ++++-- docs/connectors/sources/s3-file-source.md | 6 ++++-- quixstreams/sources/community/file/file.py | 21 ++++++++++++-------- 4 files changed, 25 insertions(+), 14 deletions(-) diff --git a/docs/connectors/sources/azure-file-source.md b/docs/connectors/sources/azure-file-source.md index b025e1064..cfda3e679 100644 --- a/docs/connectors/sources/azure-file-source.md +++ b/docs/connectors/sources/azure-file-source.md @@ -97,9 +97,11 @@ Here are some important configurations to be aware of (see [File Source API](../ **Default**: `"json"` - `compression`: what compression is used on the given files, if any (ex: `"gzip"`) **Default**: `None` -- `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. +- `replay_speed`: Produce the messages with this speed multiplier, which roughly + reflects the time "delay" between the original message producing. + Use any `float` `>= 0.0`, where `0.0` is no delay, and `1.0` is the original speed. **Note**: Time delay will only be accurate _per partition_, NOT overall. - **Default**: `True` + **Default**: 1.0 ## File hierarchy/structure diff --git a/docs/connectors/sources/local-file-source.md b/docs/connectors/sources/local-file-source.md index 89a48d7d3..3ce8d6ca5 100644 --- a/docs/connectors/sources/local-file-source.md +++ b/docs/connectors/sources/local-file-source.md @@ -78,9 +78,11 @@ Here are some important configurations to be aware of (see [File Source API](../ **Default**: `"json"` - `compression`: what compression is used on the given files, if any (ex: `"gzip"`) **Default**: `None` -- `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. +- `replay_speed`: Produce the messages with this speed multiplier, which roughly + reflects the time "delay" between the original message producing. + Use any `float` `>= 0.0`, where `0.0` is no delay, and `1.0` is the original speed. **Note**: Time delay will only be accurate _per partition_, NOT overall. - **Default**: `True` + **Default**: 1.0 ## File hierarchy/structure diff --git a/docs/connectors/sources/s3-file-source.md b/docs/connectors/sources/s3-file-source.md index c9dafa159..681063a80 100644 --- a/docs/connectors/sources/s3-file-source.md +++ b/docs/connectors/sources/s3-file-source.md @@ -104,9 +104,11 @@ Here are some important configurations to be aware of (see [File Source API](../ **Default**: `"json"` - `compression`: what compression is used on the given files, if any (ex: `"gzip"`) **Default**: `None` -- `as_replay`: Produce the messages with the original time delay between them, else as fast as possible. +- `replay_speed`: Produce the messages with this speed multiplier, which roughly + reflects the time "delay" between the original message producing. + Use any `float` `>= 0.0`, where `0.0` is no delay, and `1.0` is the original speed. **Note**: Time delay will only be accurate _per partition_, NOT overall. - **Default**: `True` + **Default**: 1.0 ## File hierarchy/structure diff --git a/quixstreams/sources/community/file/file.py b/quixstreams/sources/community/file/file.py index 3f3eee4fe..0d3f4dc84 100644 --- a/quixstreams/sources/community/file/file.py +++ b/quixstreams/sources/community/file/file.py @@ -82,7 +82,7 @@ def __init__( format: Union[Format, FormatName] = "json", origin: Origin = LocalOrigin(), compression: Optional[CompressionName] = None, - as_replay: bool = True, + replay_speed: float = 1.0, name: Optional[str] = None, shutdown_timeout: float = 10, ): @@ -94,17 +94,21 @@ def __init__( is necessary to define (compression will then be ignored). :param origin: an Origin type (defaults to reading local files). :param compression: what compression is used on the given files, if any. - :param as_replay: Produce the messages with the original time delay between them. - Otherwise, produce the messages as fast as possible. + :param replay_speed: Produce the messages with this speed multiplier, which + roughly reflects the time "delay" between the original message producing. + Use any float >= 0, where 0 is no delay, and 1 is the original speed. NOTE: Time delay will only be accurate per partition, NOT overall. :param name: The name of the Source application (Default: last folder name). :param shutdown_timeout: Time in seconds the application waits for the source to gracefully shutdown """ + if not replay_speed >= 0: + raise ValueError("`replay_speed` must be a positive value") + self._directory = Path(directory) self._origin = origin self._formatter = _get_formatter(format, compression) - self._as_replay = as_replay + self._replay_speed = replay_speed self._previous_timestamp = None self._previous_partition = None super().__init__( @@ -118,9 +122,10 @@ def _replay_delay(self, current_timestamp: int): """ if self._previous_timestamp is not None: time_diff_seconds = (current_timestamp - self._previous_timestamp) / 1000 - if time_diff_seconds > 0.01: # only sleep when diff is "big enough" - logger.debug(f"Sleeping for {time_diff_seconds} seconds...") - sleep(time_diff_seconds) + replay_diff_seconds = time_diff_seconds * self._replay_speed + if replay_diff_seconds > 0.01: # only sleep when diff is "big enough" + logger.debug(f"Sleeping for {replay_diff_seconds} seconds...") + sleep(replay_diff_seconds) self._previous_timestamp = current_timestamp def _check_file_partition_number(self, file: Path): @@ -165,7 +170,7 @@ def run(self): self._check_file_partition_number(file) filestream = self._origin.get_raw_file_stream(file) for record in self._formatter.read(filestream): - if self._as_replay and (timestamp := record.get("_timestamp")): + if timestamp := record.get("_timestamp"): self._replay_delay(timestamp) self._produce(record) self.flush() From 5a2e771cd2e201a8960f1978dc02ffd5197aa48c Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 22:14:54 -0500 Subject: [PATCH 23/24] remove azure stuff --- docs/connectors/sources/azure-file-source.md | 207 ------------------ .../community/file/origins/__init__.py | 1 - .../sources/community/file/origins/azure.py | 59 ----- 3 files changed, 267 deletions(-) delete mode 100644 docs/connectors/sources/azure-file-source.md delete mode 100644 quixstreams/sources/community/file/origins/azure.py diff --git a/docs/connectors/sources/azure-file-source.md b/docs/connectors/sources/azure-file-source.md deleted file mode 100644 index cfda3e679..000000000 --- a/docs/connectors/sources/azure-file-source.md +++ /dev/null @@ -1,207 +0,0 @@ -# Microsoft Azure File Source - -!!! info - - This is a **Community** connector. Test it before using in production. - - To learn more about differences between Core and Community connectors, see the [Community and Core Connectors](../community-and-core.md) page. - -This source reads records from files located in an Azure container path and produces -them as messages to a kafka topic using any desired `StreamingDataFrame`-based transformations. - -The resulting messages can be produced in "replay" mode, where the time between record -producing is matched as close as possible to the original. (per topic partition only). - - -## How To Install - -Install Quix Streams with the following optional dependencies: - -```bash -pip install quixstreams[azure] -``` - -## How It Works - -`FileSource` steps through each folder within the provided path and dumps each record -contained in each file as a message to a Kafka topic. Folders are navigated in -lexicographical order. - -Records are read in a streaming fashion and committed after every file, offering -[at-least-once guarantees](#processingdelivery-guarantees). - -It can handle one given file type (ex. JSONlines or Parquet) at a time, and also -supports file decompression. - -You can learn more details about the [expected kafka message format](#message-data-formatschema) below. - -## How To Use - -Azure File Source is just a special configuration of the `FileSource` connector. - -Simply provide it an `AzureOrigin` (`FileSource(origin=)`). - -Then, hand the configured `FileSource` to your `SDF` (`app.dataframe(source=)`). - -For more details around various settings, see [configuration](#configuration). - -```python -from quixstreams import Application -from quixstreams.sources.community.file import FileSource -from quixstreams.sources.community.file.origins import AzureFilesOrigin - -app = Application(broker_address="localhost:9092", auto_offset_reset="earliest") - -origin = AzureFilesOrigin( - container="", - connection_string="", -) -source = FileSource( - directory="path/to/your/topic_folder/", - origin=origin, - format="json", - compression="gzip", -) -sdf = app.dataframe(source=source).print(metadata=True) -# YOUR LOGIC HERE! - -if __name__ == "__main__": - app.run() -``` - -## Configuration - -Here are some important configurations to be aware of (see [File Source API](../../api-reference/sources.md#filesource) for all parameters). - -### Required: - -`AzureOrigin`: - -- `connection_string`: Azure client authentication string. -- `container`: Azure container name. - - -`FileSource`: - -- `directory`: a directory to recursively read through (exclude container name). - **Note**: If using alongside `FileSink`, provide the path to the topic name folder (ex: `"path/to/topic_a/"`). -- `origin`: An `AzureOrigin` instance. - - -### Optional: - -`FileSource`: - -- `format`: what format the message files are in (ex: `"json"`, `"parquet"`). - **Advanced**: can optionally provide a `Format` instance (`compression` will then be ignored). - **Default**: `"json"` -- `compression`: what compression is used on the given files, if any (ex: `"gzip"`) - **Default**: `None` -- `replay_speed`: Produce the messages with this speed multiplier, which roughly - reflects the time "delay" between the original message producing. - Use any `float` `>= 0.0`, where `0.0` is no delay, and `1.0` is the original speed. - **Note**: Time delay will only be accurate _per partition_, NOT overall. - **Default**: 1.0 - -## File hierarchy/structure - -The File Source expects a folder structure like so: - -``` - my_sinked_topics/ - ├── topic_a/ # topic name (use this path to File Source!) - │ ├── 0/ # topic partition number - │ │ ├── 0000.ext # formatted offset files (ex: JSON) - │ │ └── 0011.ext - │ └── 1/ - │ ├── 0003.ext - │ └── 0016.ext - └── topic_b/ - └── etc... -``` - -## Message Data Format/Schema - -The expected file schema largely depends on the chosen -file format. - -For easiest use (especially alongside [`FileSink`](../sinks/file-sink.md)), -you can follow these patterns: - -### Row-based Formats (ex: JSON) - -Files should have records with the following fields, with `_value` being a -JSON-deserializable item: - - - `_key` - - `_value` - - `_timestamp` - - -This will result in the following Kafka message format for `Application`: - -- Message `key` will be the record `_key` as `bytes`. -- Message `value` will be the record `_value` as a `json`/`dict` -- Message `timestamp` will be the record `_timestamp` (ms). - -### Columnar Formats (ex: Parquet) -These do not expect an explicit `value` field; instead all columns should be included -individually while including `_key` and `_timestamp`: - - - `_key` - - `_timestamp` - - `field_a` - - `field_b` - etc... - - -This will result in the following Kafka message format for `Application`: - -- Message `key` will be the record `_key` as `bytes`. -- Message `value` will be every record field except `_key` and `_timestamp` packed as a `json`/`dict` -- Message `timestamp` will be the record `_timestamp` (ms). - - -### Custom Schemas (Advanced) - -If the original files are not formatted as expected, custom loaders can be configured -on some `Format` classes (ex: `JsonFormat`) which can be handed to `FileSource(format=)`. - -Formats can be imported from `quixstreams.sources.community.file.formats`. - -## Processing/Delivery Guarantees - -This Source offers "at-least-once" guarantees with message delivery: messages are -guaranteed to be committed when a file is finished processing. - -However, it does not save any state/position: an unhandled exception will cause the -`Application` to fail, and rerunning the `Application` will begin processing from the -beginning (reproducing all previously processed messages). - -## Topic - -The default topic will have a partition count that reflects the partition count found -within the provided topic's folder structure. - -The default topic name the Application dumps to is based on the last folder name of -the `FileSource` `directory` as: `source__`. - - -## Testing Locally - -Rather than connect to Azure, you can alternatively test your application using a local -emulated Azure host via Docker: - -1. Execute in terminal: - - ```bash - docker run --rm -d --name azurite \ - -p 10000:10000 \ - mcr.microsoft.com/azure-storage/azurite:latest - ``` - -2. Set `connection_string` for `AzureOrigin` to: - -```python -"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" -``` diff --git a/quixstreams/sources/community/file/origins/__init__.py b/quixstreams/sources/community/file/origins/__init__.py index dded4c541..01248aef5 100644 --- a/quixstreams/sources/community/file/origins/__init__.py +++ b/quixstreams/sources/community/file/origins/__init__.py @@ -1,4 +1,3 @@ # ruff: noqa: F403 -from .azure import * from .local import * from .s3 import * diff --git a/quixstreams/sources/community/file/origins/azure.py b/quixstreams/sources/community/file/origins/azure.py deleted file mode 100644 index dbfb0c638..000000000 --- a/quixstreams/sources/community/file/origins/azure.py +++ /dev/null @@ -1,59 +0,0 @@ -import os -from io import BytesIO -from pathlib import Path -from typing import Generator - -from .base import Origin - -try: - from azure.storage.blob import BlobServiceClient - from azure.storage.blob._container_client import ContainerClient -except ImportError as exc: - raise ImportError( - f"Package {exc.name} is missing: " - 'run "pip install quixstreams[azure]" to use AzureOrigin' - ) from exc - -__all__ = ("AzureFilesOrigin",) - - -class AzureFilesOrigin(Origin): - def __init__( - self, - connection_string: str, - container: str, - ): - """ - :param connection_string: Azure client authentication string. - :param container: Azure container name. - """ - self.root_location = container - self._client = self._get_client(connection_string) - - def _get_client(self, auth: str) -> ContainerClient: - blob_client = BlobServiceClient.from_connection_string(auth) - return blob_client.get_container_client(self.root_location) - - def file_collector(self, filepath: Path) -> Generator[Path, None, None]: - data = self._client.list_blob_names(name_starts_with=str(filepath)) - for page in data.by_page(): - for item in page: - yield Path(item) - - def get_folder_count(self, directory: Path) -> int: - """ - This is a simplified version of the recommended way to retrieve folder - names based on the azure SDK docs examples. - """ - path = f"{directory}/" - folders = set() - for blob in self._client.list_blobs(name_starts_with=path): - relative_dir = os.path.dirname(os.path.relpath(blob.name, path)) - if relative_dir and ("/" not in relative_dir): - folders.add(relative_dir) - return len(folders) - - def get_raw_file_stream(self, filepath: Path) -> BytesIO: - blob_client = self._client.get_blob_client(str(filepath)) - data = blob_client.download_blob().readall() - return BytesIO(data) From e248f0c1d794fd28b3cebed74b25bce7aca8ff5c Mon Sep 17 00:00:00 2001 From: Tim Sawicki Date: Mon, 2 Dec 2024 22:17:44 -0500 Subject: [PATCH 24/24] fix a doc ref --- docs/connectors/sources/local-file-source.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/connectors/sources/local-file-source.md b/docs/connectors/sources/local-file-source.md index 3ce8d6ca5..8626c3c42 100644 --- a/docs/connectors/sources/local-file-source.md +++ b/docs/connectors/sources/local-file-source.md @@ -53,7 +53,7 @@ source = FileSource( directory="/path/to/my/topic_folder", format="json", compression="gzip", - as_replay=True, + replay_speed=1.0, ) sdf = app.dataframe(source=source).print(metadata=True) # YOUR LOGIC HERE!