From 59a646d8bc3d3259fb2a70e1f702e8d2bc4cf24f Mon Sep 17 00:00:00 2001 From: Sergey Kulik <104143901+zxqfd555-pw@users.noreply.github.com> Date: Wed, 29 Jan 2025 14:31:13 +0100 Subject: [PATCH] repurpose persistent_id to use it as a connector name (#8073) GitOrigin-RevId: cb259294265cae0f74921a90e7c0f147114b466d --- CHANGELOG.md | 3 + .../60.deployment/55.persistence.md | 16 +- .../60.deployment/60.persistence_recovery.md | 12 +- .../65.persistence_restart_with_new_data.md | 8 +- integration_tests/kafka/test_backfilling.py | 2 +- integration_tests/kafka/test_simple.py | 4 +- integration_tests/s3/test_s3_generic.py | 8 +- integration_tests/wordcount/pw_wordcount.py | 2 +- python/pathway/debug/__init__.py | 25 ++- python/pathway/demo/__init__.py | 13 +- python/pathway/engine.pyi | 9 +- python/pathway/internals/column.py | 2 +- python/pathway/internals/datasink.py | 2 + python/pathway/internals/datasource.py | 2 + .../graph_runner/expression_evaluator.py | 2 +- .../graph_runner/operator_handler.py | 2 + python/pathway/internals/table.py | 8 +- .../pathway/internals/table_subscription.py | 4 + python/pathway/io/_subscribe.py | 5 + python/pathway/io/_utils.py | 37 +++- python/pathway/io/airbyte/__init__.py | 9 +- python/pathway/io/bigquery/__init__.py | 28 ++- python/pathway/io/csv/__init__.py | 18 +- python/pathway/io/debezium/__init__.py | 17 +- python/pathway/io/deltalake/__init__.py | 21 +- python/pathway/io/elasticsearch/__init__.py | 12 +- python/pathway/io/fs/__init__.py | 33 ++- python/pathway/io/gdrive/__init__.py | 16 +- python/pathway/io/http/__init__.py | 12 +- python/pathway/io/http/_server.py | 61 +++--- python/pathway/io/http/_streaming.py | 2 +- python/pathway/io/iceberg/__init__.py | 21 +- python/pathway/io/jsonlines/__init__.py | 66 +++--- python/pathway/io/kafka/__init__.py | 79 +++---- python/pathway/io/logstash/__init__.py | 17 +- python/pathway/io/minio/__init__.py | 21 +- python/pathway/io/mongodb/__init__.py | 12 +- python/pathway/io/nats/__init__.py | 24 ++- python/pathway/io/null/__init__.py | 5 +- python/pathway/io/plaintext/__init__.py | 60 +++--- python/pathway/io/postgres/__init__.py | 19 +- python/pathway/io/pubsub/__init__.py | 24 ++- python/pathway/io/pyfilesystem/__init__.py | 68 +++--- python/pathway/io/python/__init__.py | 28 +-- python/pathway/io/redpanda/__init__.py | 26 +-- python/pathway/io/s3/__init__.py | 49 +++-- python/pathway/io/s3_csv/__init__.py | 17 +- python/pathway/io/sqlite/__init__.py | 7 +- .../pathway/stdlib/utils/async_transformer.py | 3 +- python/pathway/tests/cli/replay.py | 2 +- python/pathway/tests/test_deduplicate.py | 20 +- python/pathway/tests/test_io.py | 79 +++++-- python/pathway/tests/test_persistence.py | 8 +- .../xpacks/connectors/sharepoint/__init__.py | 4 +- src/connectors/data_lake/delta.rs | 17 +- src/connectors/data_lake/iceberg.rs | 26 ++- src/connectors/data_lake/mod.rs | 2 + src/connectors/data_lake/writer.rs | 4 + src/connectors/data_storage.rs | 194 ++++++++---------- src/connectors/mod.rs | 21 +- src/connectors/monitoring.rs | 6 +- src/connectors/posix_like.rs | 20 +- src/connectors/scanner/filesystem.rs | 4 + src/connectors/scanner/mod.rs | 1 + src/connectors/scanner/s3.rs | 4 + src/engine/dataflow.rs | 106 ++++++---- src/engine/dataflow/persist.rs | 20 +- src/engine/error.rs | 4 - src/engine/graph.rs | 29 ++- src/persistence/mod.rs | 4 +- src/python_api.rs | 176 ++++++++-------- tests/integration/helpers.rs | 19 +- tests/integration/test_arrow.rs | 4 + tests/integration/test_bytes.rs | 2 +- .../test_connector_field_defaults.rs | 16 +- tests/integration/test_debezium.rs | 4 +- tests/integration/test_deltalake.rs | 1 - tests/integration/test_dsv.rs | 16 +- tests/integration/test_dsv_dir.rs | 14 +- tests/integration/test_jsonlines.rs | 22 +- tests/integration/test_metadata.rs | 16 +- tests/integration/test_seek.rs | 60 ++++-- 82 files changed, 1072 insertions(+), 794 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cdbc3cd2..b38e9fcf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,8 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm - methods `pw.io.postgres.write` and `pw.io.postgres.write_snapshot` now accept an additional argument `init_mode`, which allows initializing the table before writing. - `pw.io.deltalake.read` now supports serialization and deserialization for all Pathway data types. - New parser `pathway.xpacks.llm.parsers.DoclingParser` supporting parsing of pdfs with tables and images. +- Output connectors now include an optional `name` parameter. If provided, this name will appear in logs and monitoring dashboards. +- Automatic naming for input and output connectors has been enhanced. ### Changed - **BREAKING**: `pw.io.deltalake.read` now requires explicit specification of primary key fields. @@ -18,6 +20,7 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm - **BREAKING**: When using delay in temporal behavior, current time is updated immediately, not in the next batch. - **BREAKING**: The `Pointer` type is now serialized to Delta Tables as raw bytes. - `pw.io.kafka.write` now allows to specify `key` and `headers` for JSON and CSV data formats. +- `persistent_id` parameter in connectors has been renamed to `name`. This new `name` parameter allows you to assign names to connectors, which will appear in logs and monitoring dashboards. ### Fixed - `generate_class` method in `Schema` now correctly renders columns of `UnionType` and `None` types. diff --git a/docs/2.developers/4.user-guide/60.deployment/55.persistence.md b/docs/2.developers/4.user-guide/60.deployment/55.persistence.md index 89b6d8c0..792232cd 100644 --- a/docs/2.developers/4.user-guide/60.deployment/55.persistence.md +++ b/docs/2.developers/4.user-guide/60.deployment/55.persistence.md @@ -64,24 +64,24 @@ Both metadata and snapshot storages are configurable with the class [`pw.persist The config itself is created via constructor, where it accepts `backend` argument of the type `pw.persistence.Backend`, and optionally the snapshot interval given in milliseconds. -This, however, is not the only thing needed for persistence to work, and it moves us to the persistent ids. +This, however, is not the only thing needed for persistence to work, and it moves us to the unique names. -### Persistent IDs +### Unique names -To persist certain input sources, Pathway requires them to have the `persistent_id` parameter. This identifier denotes the factual data source, so it is also expected not to be changed over the reruns of the program. +To persist certain input sources, Pathway uses the `name` parameter. This identifier denotes the factual data source, so it is also expected not to be changed over the reruns of the program. -The motivation behind the persistent id is that the data source can be changed as much as it's needed for the computations as long as it contains the data with the same schema. For example, it is possible that: +The motivation behind the unique names usage is that the data source can be changed as much as it's needed for the computations as long as it contains the data with the same schema. For example, it is possible that: * a data format in the source is changed. For instance, JSON was used, however, the newer entries are in CSV; * a path to the data source is changed. For instance, the logs that are parsed by Pathway are now stored on a different volume; * some fields are renamed in the data source. As an example, it is possible that the field "date" was renamed to "datetime" to have a more accurate name. -All in all, the variety of changes can be huge. Still, by using the same `persistent_id` the engine knows that the data still corresponds to a certain table. +All in all, the variety of changes can be huge. Still, by using the same `name` the engine knows that the data still corresponds to a certain table. -These IDs can be assigned in two different ways. The first one is automatic generation. It will assign the IDs to the sources based on the order in which they are added. For example, if the program first reads a dataset from the CSV and then it reads the stream of events from Kafka, it will have two persistent IDs automatically generated, where the first one will point to the dataset and the second one will point to the stream of events. Please note that this approach is fine in case the code is not going to be updated. Otherwise, the sources can be changed which will incur the generation of different persistent IDs, not corresponding to the old ones. +These IDs can be assigned in two different ways. The first one is automatic generation. It will assign the IDs to the sources based on the order in which they are added. For example, if the program first reads a dataset from the CSV and then it reads the stream of events from Kafka, it will have two unique names automatically generated, where the first one will point to the dataset and the second one will point to the stream of events. Please note that this approach is fine in case the code is not going to be updated. Otherwise, the sources can be changed which will incur the generation of different unique names, not corresponding to the old ones. -The second way can be a bit more difficult, but it allows more flexibility: the persistent IDs can be assigned manually in the input connector. To do so, one needs to specify the string parameter `persistent_id`. For example, the word count program above relied on the automatic `persistent_id` generation. However, it would also be possible to specify it explicitly this way: +The second way can be a bit more difficult, but it allows more flexibility: the unique names can be assigned manually in the input connector. To do so, one needs to specify the string parameter `name`. For example, the word count program above relied on the automatic `name` generation. However, it would also be possible to specify it explicitly this way: ```python -words = pw.io.csv.read("inputs/", schema=InputSchema, persistent_id="words_source") +words = pw.io.csv.read("inputs/", schema=InputSchema, name="words_source") ``` ## Requirements and assumptions diff --git a/docs/2.developers/4.user-guide/60.deployment/60.persistence_recovery.md b/docs/2.developers/4.user-guide/60.deployment/60.persistence_recovery.md index 74e68306..96e63f10 100644 --- a/docs/2.developers/4.user-guide/60.deployment/60.persistence_recovery.md +++ b/docs/2.developers/4.user-guide/60.deployment/60.persistence_recovery.md @@ -234,18 +234,18 @@ backend = pw.persistence.Backend.filesystem("./PStorage") persistence_config = pw.persistence.Config(backend) ``` -## Persistent IDs +## Unique Names -The second (and optional) thing we need to do is the persistent ID assignment. The persistent IDs are required for the engine to match the data sources between different runs. +The second (and optional) thing we need to do is unique names assignment. These unique names are required for the engine to match the data sources between different runs. -In principle, the persistent ID assignment can be done automatically by the engine. In this case, it will assign the persistent IDs to the sources in the order of their appearance and construction. However, this is not generally recommended if you need to change your Pathway program and the data source in the future. +In principle, this assignment can be done automatically by the engine. In this case, it will assign unique names to the sources in the order of their appearance and construction. However, this is not generally recommended if you need to change your Pathway program and the data source in the future. -For the sake of completeness, in this tutorial, we will demonstrate the manual persistent ID assignment. The only difference from the non-persistent variant of the input is the parameter `persistent_id` which should be passed to the `pw.io.csv.read` method. So, if we name the data source `words_data_source` the assignment may look as follows: +For the sake of completeness, in this tutorial, we will demonstrate the manual unique names assignment. The only difference from the non-persistent variant of the input is the parameter `name` which should be passed to the `pw.io.csv.read` method. So, if we name the data source `words_data_source` the assignment may look as follows: ```python pw.io.csv.read( ..., - persistent_id="words_data_source" + name="words_data_source" ) ``` @@ -269,7 +269,7 @@ if __name__ == "__main__": "inputs/", schema=InputSchema, autocommit_duration_ms=10, - persistent_id="words_input_source", # Changed: now persistent_id is assigned here + name="words_input_source", # Changed: now name is assigned here ) word_counts = words.groupby(words.word).reduce(words.word, count=pw.reducers.count()) pw.io.jsonlines.write(word_counts, "result.jsonlines") diff --git a/docs/2.developers/4.user-guide/60.deployment/65.persistence_restart_with_new_data.md b/docs/2.developers/4.user-guide/60.deployment/65.persistence_restart_with_new_data.md index da777a08..d01dd0c0 100644 --- a/docs/2.developers/4.user-guide/60.deployment/65.persistence_restart_with_new_data.md +++ b/docs/2.developers/4.user-guide/60.deployment/65.persistence_restart_with_new_data.md @@ -172,9 +172,9 @@ Logs may arrive at varying intervals, with a cron-like job delivering data every [Persistence](/developers/api-docs/persistence-api/) can help here. The idea is to store the state of previous calculations so that when new files are added later, there's no need to start from scratch. -So you need to do two things. First, you need to "name" input sources by assigning persistent IDs to them. This way, when the program recovers, it can accurately match the operator with the correct data dump. +So you need to do two things. First, you need to "name" input sources by assigning uniqie names to them. This way, when the program recovers, it can accurately match the operator with the correct data dump. -Here, it is optional because the data processing pipeline doesn't change. It will be shown for the fullness of the example that the only difference is the parameter `persistent_id`: +Here, it is optional because the data processing pipeline doesn't change. It will be shown for the fullness of the example that the only difference is the parameter `name`: ```python @@ -182,7 +182,7 @@ access_entries = pw.io.csv.read( "logs/", schema=InputSchema, mode="static", - persistent_id="logs" + name="logs" ) ``` @@ -225,7 +225,7 @@ if __name__ == "__main__": "logs/", mode="static", schema=InputSchema, - persistent_id="logs", # Change: persistent ID assigned + name="logs", # Change: unique name assigned ) sessions = table.windowby( pw.this.access_time, diff --git a/integration_tests/kafka/test_backfilling.py b/integration_tests/kafka/test_backfilling.py index dd2b1a4d..10d22bc7 100644 --- a/integration_tests/kafka/test_backfilling.py +++ b/integration_tests/kafka/test_backfilling.py @@ -132,7 +132,7 @@ def run_backfilling_program( topic=kafka_context.input_topic, format="plaintext", autocommit_duration_ms=5, - persistent_id="1", + name="1", ), kwargs={"persistence_config": persistence_config}, ) diff --git a/integration_tests/kafka/test_simple.py b/integration_tests/kafka/test_simple.py index 0c4f84b8..70a8d6c7 100644 --- a/integration_tests/kafka/test_simple.py +++ b/integration_tests/kafka/test_simple.py @@ -396,7 +396,7 @@ def test_kafka_recovery(tmp_path: pathlib.Path, kafka_context: KafkaTestContext) value_columns=["v"], primary_key=["k"], autocommit_duration_ms=100, - persistent_id="1", + name="1", ) pw.io.csv.write(table, tmp_path / "output.csv") @@ -438,7 +438,7 @@ def test_kafka_recovery(tmp_path: pathlib.Path, kafka_context: KafkaTestContext) value_columns=["v"], primary_key=["k"], autocommit_duration_ms=100, - persistent_id="1", + name="1", ) pw.io.csv.write(table, tmp_path / "output_backfilled.csv") diff --git a/integration_tests/s3/test_s3_generic.py b/integration_tests/s3/test_s3_generic.py index 5258469f..e5aa6c10 100644 --- a/integration_tests/s3/test_s3_generic.py +++ b/integration_tests/s3/test_s3_generic.py @@ -30,7 +30,7 @@ def test_s3_backfilling(snapshot_access, tmp_path: pathlib.Path, s3_path: str): value_columns=["key", "value"], mode="static", autocommit_duration_ms=1000, - persistent_id="1", + name="1", ) pw.io.csv.write(table, str(tmp_path / "output.csv")) pw.run( @@ -50,7 +50,7 @@ def test_s3_backfilling(snapshot_access, tmp_path: pathlib.Path, s3_path: str): value_columns=["key", "value"], mode="static", autocommit_duration_ms=1000, - persistent_id="1", + name="1", ) pw.io.csv.write(table, str(tmp_path / "output_backfilled.csv")) pw.run( @@ -74,7 +74,7 @@ def test_s3_backfilling(snapshot_access, tmp_path: pathlib.Path, s3_path: str): value_columns=["key", "value"], mode="static", autocommit_duration_ms=1000, - persistent_id="1", + name="1", ) pw.io.csv.write(table, str(output_path)) pw.run( @@ -120,7 +120,7 @@ class InputSchema(pw.Schema): format="json", schema=InputSchema, mode="static", - persistent_id="1", + name="1", ) pw.io.jsonlines.write(table, str(output_path)) pw.run( diff --git a/integration_tests/wordcount/pw_wordcount.py b/integration_tests/wordcount/pw_wordcount.py index 092ba987..b9b9a510 100755 --- a/integration_tests/wordcount/pw_wordcount.py +++ b/integration_tests/wordcount/pw_wordcount.py @@ -55,7 +55,7 @@ class InputSchema(pw.Schema): schema=InputSchema, format="json", mode=args.mode, - persistent_id="1", + name="1", autocommit_duration_ms=100, ) result = words.groupby(words.word).reduce( diff --git a/python/pathway/debug/__init__.py b/python/pathway/debug/__init__.py index 88a99277..3716a752 100644 --- a/python/pathway/debug/__init__.py +++ b/python/pathway/debug/__init__.py @@ -488,17 +488,17 @@ def run(self): class StreamGenerator: - _persistent_id = itertools.count() + _unique_name = itertools.count() events: dict[tuple[str, int], list[api.SnapshotEvent]] = {} - def _get_next_persistent_id(self) -> str: - return str(f"_stream_generator_{next(self._persistent_id)}") + def _get_next_unique_name(self) -> str: + return str(f"_stream_generator_{next(self._unique_name)}") def _advance_time_for_all_workers( - self, persistent_id: str, workers: Iterable[int], timestamp: int + self, unique_name: str, workers: Iterable[int], timestamp: int ): for worker in workers: - self.events[(persistent_id, worker)].append( + self.events[(unique_name, worker)].append( api.SnapshotEvent.advance_time(timestamp) ) @@ -519,10 +519,10 @@ def _table_from_dict( batches: dictionary with specified batches to be put in the table schema: schema of the table """ - persistent_id = self._get_next_persistent_id() + unique_name = self._get_next_unique_name() workers = {worker for batch in batches.values() for worker in batch} for worker in workers: - self.events[(persistent_id, worker)] = [] + self.events[(unique_name, worker)] = [] timestamps = set(batches.keys()) @@ -541,24 +541,23 @@ def _table_from_dict( batches = {2 * timestamp: batches[timestamp] for timestamp in batches} for timestamp in sorted(batches): - self._advance_time_for_all_workers(persistent_id, workers, timestamp) + self._advance_time_for_all_workers(unique_name, workers, timestamp) batch = batches[timestamp] for worker, changes in batch.items(): for diff, key, values in changes: if diff == 1: event = api.SnapshotEvent.insert(key, values) - self.events[(persistent_id, worker)] += [event] * diff + self.events[(unique_name, worker)] += [event] * diff elif diff == -1: event = api.SnapshotEvent.delete(key, values) - self.events[(persistent_id, worker)] += [event] * (-diff) + self.events[(unique_name, worker)] += [event] * (-diff) else: raise ValueError("only diffs of 1 and -1 are supported") return read( - _EmptyConnectorSubject(), - persistent_id=persistent_id, + _EmptyConnectorSubject(datasource_name="debug.stream-generator"), + name=unique_name, schema=schema, - name="debug.stream-generator", ) def table_from_list_of_batches_by_workers( diff --git a/python/pathway/demo/__init__.py b/python/pathway/demo/__init__.py index 0c84645a..979460d7 100644 --- a/python/pathway/demo/__init__.py +++ b/python/pathway/demo/__init__.py @@ -32,7 +32,7 @@ def generate_custom_stream( nb_rows: int | None = None, autocommit_duration_ms: int = 1000, input_rate: float = 1.0, - persistent_id: str | None = None, + name: str | None = None, ) -> pw.Table: """Generates a data stream. @@ -104,12 +104,11 @@ def _get_row(i): time.sleep(1.0 / input_rate) table = pw.io.python.read( - FileStreamSubject(), + FileStreamSubject(datasource_name="demo.custom-stream"), schema=schema, format="json", autocommit_duration_ms=autocommit_duration_ms, - persistent_id=persistent_id, - name="demo.custom-stream", + name=name, ) return table @@ -247,11 +246,10 @@ def run(self): time.sleep(1.0 / input_rate) return pw.io.python.read( - FileStreamSubject(), + FileStreamSubject(datasource_name="demo.replay-csv"), schema=schema.with_types(**{name: str for name in schema.column_names()}), autocommit_duration_ms=autocommit_ms, format="json", - name="demo.replay-csv", ).cast_to_types(**schema.typehints()) @@ -331,9 +329,8 @@ def run(self): self.next_json(values) return pw.io.python.read( - FileStreamSubject(), + FileStreamSubject(datasource_name="demo.replay-csv-with-time"), schema=schema.with_types(**{name: str for name in schema.column_names()}), autocommit_duration_ms=autocommit_ms, format="json", - name="demo.replay-csv-with-time", ).cast_to_types(**schema.typehints()) diff --git a/python/pathway/engine.pyi b/python/pathway/engine.pyi index a93616af..6864b5fd 100644 --- a/python/pathway/engine.pyi +++ b/python/pathway/engine.pyi @@ -97,6 +97,7 @@ class ConnectorProperties: commit_duration_ms: int | None = None unsafe_trusted_ids: bool | None = False column_properties: list[ColumnProperties] = [] + unique_name: str | None = None class Column: """A Column holds data and conceptually is a Dict[Universe elems, dt] @@ -596,7 +597,7 @@ class Scope: grouping_columns: list[ColumnPath], reduced_columns: list[ColumnPath], combine: Callable[[Any, Any], Any], - persistent_id: str | None, + unique_name: str | None, table_properties: TableProperties, ) -> Table: ... def ix_table( @@ -671,6 +672,7 @@ class Scope: on_change: Callable, on_time_end: Callable, on_end: Callable, + unique_name: str | None = None, ): ... def output_table( self, @@ -678,6 +680,7 @@ class Scope: column_paths: Iterable[ColumnPath], data_sink: DataStorage, data_format: DataFormat, + unique_name: str | None = None, ): ... def export_table( self, table: Table, column_paths: Iterable[ColumnPath] @@ -752,7 +755,7 @@ class DataStorage: elasticsearch_params: ElasticSearchParams | None parallel_readers: int | None python_subject: PythonSubject | None - persistent_id: str | None + unique_name: str | None max_batch_size: int | None object_pattern: str mock_events: dict[tuple[str, int], list[SnapshotEvent]] | None @@ -824,7 +827,7 @@ class SnapshotEvent: FINISHED: SnapshotEvent class LocalBinarySnapshotWriter: - def __init__(self, path: str, persistent_id: str, worker_id: int): ... + def __init__(self, path: str, unique_name: str, worker_id: int): ... def write(self, events: list[SnapshotEvent]): ... class TelemetryConfig: diff --git a/python/pathway/internals/column.py b/python/pathway/internals/column.py index 6bfd93af..06b7f80e 100644 --- a/python/pathway/internals/column.py +++ b/python/pathway/internals/column.py @@ -544,7 +544,7 @@ class DeduplicateContext(Context): instance: tuple[ColumnWithExpression, ...] acceptor: Callable[[Any, Any], bool] orig_id_column: IdColumn - persistent_id: str | None + unique_name: str | None def column_dependencies_internal(self) -> Iterable[Column]: return (self.value,) + self.instance diff --git a/python/pathway/internals/datasink.py b/python/pathway/internals/datasink.py index 7a201137..ef2b2a30 100644 --- a/python/pathway/internals/datasink.py +++ b/python/pathway/internals/datasink.py @@ -20,6 +20,7 @@ class GenericDataSink(DataSink): datastorage: api.DataStorage dataformat: api.DataFormat datasink_name: str + unique_name: str | None @property def name(self) -> str: @@ -33,6 +34,7 @@ class CallbackDataSink(DataSink): on_end: Callable[[], None] skip_persisted_batch: bool skip_errors: bool + unique_name: str | None @dataclass(frozen=True) diff --git a/python/pathway/internals/datasource.py b/python/pathway/internals/datasource.py index 35485a22..a1df8054 100644 --- a/python/pathway/internals/datasource.py +++ b/python/pathway/internals/datasource.py @@ -17,6 +17,7 @@ class DataSourceOptions: commit_duration_ms: int | None = None unsafe_trusted_ids: bool | None = False + unique_name: str | None = None @dataclass(frozen=True, kw_only=True) @@ -39,6 +40,7 @@ def connector_properties(self) -> api.ConnectorProperties: commit_duration_ms=self.data_source_options.commit_duration_ms, unsafe_trusted_ids=self.data_source_options.unsafe_trusted_ids, column_properties=columns, + unique_name=self.data_source_options.unique_name, ) def get_effective_schema(self) -> type[Schema]: diff --git a/python/pathway/internals/graph_runner/expression_evaluator.py b/python/pathway/internals/graph_runner/expression_evaluator.py index 2dadd93d..f558a79a 100644 --- a/python/pathway/internals/graph_runner/expression_evaluator.py +++ b/python/pathway/internals/graph_runner/expression_evaluator.py @@ -1251,7 +1251,7 @@ def is_different_with_state( instance_paths, reduced_columns_paths, is_different_with_state, - self.context.persistent_id, + self.context.unique_name, properties, ) diff --git a/python/pathway/internals/graph_runner/operator_handler.py b/python/pathway/internals/graph_runner/operator_handler.py index 80dafd4e..0989b84c 100644 --- a/python/pathway/internals/graph_runner/operator_handler.py +++ b/python/pathway/internals/graph_runner/operator_handler.py @@ -187,6 +187,7 @@ def _run( column_paths=column_paths, data_sink=datasink.datastorage, data_format=datasink.dataformat, + unique_name=datasink.unique_name, ) elif isinstance(datasink, CallbackDataSink): self.scope.subscribe_table( @@ -197,6 +198,7 @@ def _run( on_end=datasink.on_end, skip_persisted_batch=datasink.skip_persisted_batch, skip_errors=datasink.skip_errors, + unique_name=datasink.unique_name, ) elif isinstance(datasink, ExportDataSink): exported_table = self.scope.export_table( diff --git a/python/pathway/internals/table.py b/python/pathway/internals/table.py index c778beb7..2e168dc1 100644 --- a/python/pathway/internals/table.py +++ b/python/pathway/internals/table.py @@ -1067,7 +1067,7 @@ def deduplicate( value: expr.ColumnExpression | Value, instance: expr.ColumnExpression | None = None, acceptor: Callable[[T, T], bool], - persistent_id: str | None = None, + name: str | None = None, ) -> Table: """Deduplicates rows in `self` on `value` column using acceptor function. @@ -1080,10 +1080,10 @@ def deduplicate( values in this column, deduplication will be performed separately. Defaults to None. acceptor: callback telling whether two values are different. - persistent_id: (unstable) An identifier, under which the state of the table + name: An identifier, under which the state of the table will be persisted or ``None``, if there is no need to persist the state of this table. When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of + was saved for their ``name``. This way it's possible to configure the start of computations from the moment they were terminated last time. Returns: @@ -1155,7 +1155,7 @@ def deduplicate( (instance_col,), acceptor, self._id_column, - persistent_id, + name, ) return self._table_with_context(context) diff --git a/python/pathway/internals/table_subscription.py b/python/pathway/internals/table_subscription.py index 28fe6077..6726c172 100644 --- a/python/pathway/internals/table_subscription.py +++ b/python/pathway/internals/table_subscription.py @@ -83,6 +83,7 @@ def subscribe( on_time_end: OnTimeEndCallback = lambda time: None, on_end: OnFinishCallback = lambda: None, skip_errors: bool = True, + name: str | None = None, ) -> None: """ Calls a callback function on_change on every change happening in table. This method @@ -103,6 +104,8 @@ def subscribe( on_time_end: the callback function to be called on each closed time of computation. on_end: the callback function to be called when the stream of changes ends. skip_errors: whether to skip rows containing errors + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None """ @@ -142,5 +145,6 @@ def on_change_wrapper( on_end=on_end, skip_persisted_batch=skip_persisted_batch, skip_errors=skip_errors, + unique_name=name, ), ) diff --git a/python/pathway/io/_subscribe.py b/python/pathway/io/_subscribe.py index 4e9cd54f..f9d622a5 100644 --- a/python/pathway/io/_subscribe.py +++ b/python/pathway/io/_subscribe.py @@ -15,6 +15,8 @@ def subscribe( on_change: OnChangeCallback, on_end: OnFinishCallback = lambda: None, on_time_end: OnTimeEndCallback = lambda time: None, + *, + name: str | None = None, ): """ Calls a callback function on_change on every change happening in table. @@ -28,6 +30,8 @@ def subscribe( names key, row, time and is_addition respectively. on_end: the callback to be called when the stream of changes ends. on_time_end: the callback function to be called on each closed time of computation. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -66,4 +70,5 @@ def subscribe( on_change=on_change, on_time_end=on_time_end, on_end=on_end, + name=name, ) diff --git a/python/pathway/io/_utils.py b/python/pathway/io/_utils.py index 46bc3e3b..c8e37ea8 100644 --- a/python/pathway/io/_utils.py +++ b/python/pathway/io/_utils.py @@ -101,8 +101,13 @@ def check_deprecated_kwargs( stacklevel=stacklevel + 1, ) kwargs.pop(kwarg_name) - if kwargs: - unexpected_arg_names = ", ".join(repr(arg) for arg in kwargs.keys()) + deprecated_kwargs_with_custom_report = ["persistent_id"] + unexpected_args_list = [] + for kwarg in kwargs.keys(): + if kwarg not in deprecated_kwargs_with_custom_report: + unexpected_args_list.append(kwarg) + if unexpected_args_list: + unexpected_arg_names = ", ".join(repr(arg) for arg in unexpected_args_list) raise TypeError(f"Got unexpected keyword arguments: {unexpected_arg_names}") @@ -136,9 +141,9 @@ class CsvParserSettings: escape: What character to use for escaping fields in CSV. enable_double_quote_escapes: Enable escapes of double quotes. enable_quoting: Enable quoting for the fields. - comment_character: If specified, the lines starting with the comment \ -character will be treated as comments and therefore, will be ignored by \ -parser + comment_character: If specified, the lines starting with the comment + character will be treated as comments and therefore, will be ignored by + parser """ def __init__( @@ -371,7 +376,6 @@ def construct_s3_data_storage( *, downloader_threads_count: int | None = None, csv_settings: CsvParserSettings | None = None, - persistent_id: str | None = None, ): if format == "csv": return api.DataStorage( @@ -381,7 +385,6 @@ def construct_s3_data_storage( csv_parser_settings=csv_settings.api_settings if csv_settings else None, downloader_threads_count=downloader_threads_count, mode=internal_connector_mode(mode), - persistent_id=persistent_id, ) else: return api.DataStorage( @@ -391,7 +394,6 @@ def construct_s3_data_storage( mode=internal_connector_mode(mode), read_method=internal_read_method(format), downloader_threads_count=downloader_threads_count, - persistent_id=persistent_id, ) @@ -522,3 +524,22 @@ def add_column_reference_to_extract( field_indices[column_name] = index_in_new_table selection_list.append(column_reference) return index_in_new_table + + +def _get_unique_name( + name: str | None, kwargs: dict[str, Any], stacklevel: int = 6 +) -> str: + deprecated_name = kwargs.get("persistent_id") + if name is not None: + if deprecated_name is not None: + raise ValueError( + "'persistent_id' and 'name' should not be used together. Please use 'name' only." + ) + return name + if deprecated_name is not None: + warnings.warn( + "'persistent_id' is deprecated. Please use 'name' instead.", + DeprecationWarning, + stacklevel=stacklevel, + ) + return deprecated_name diff --git a/python/pathway/io/airbyte/__init__.py b/python/pathway/io/airbyte/__init__.py index 65da9b8f..ce98dc0b 100644 --- a/python/pathway/io/airbyte/__init__.py +++ b/python/pathway/io/airbyte/__init__.py @@ -11,6 +11,7 @@ import yaml from pathway.internals.schema import Schema +from pathway.io._utils import _get_unique_name from pathway.io.python import read as python_connector_read from pathway.optional_import import optional_imports from pathway.third_party.airbyte_serverless.executable_runner import ( @@ -116,7 +117,8 @@ def read( gcp_job_name: str | None = None, enforce_method: str | None = None, refresh_interval_ms: int = 60000, - persistent_id: int | None = None, + name: str | None = None, + **kwargs, ): """ Reads a table with a free tier Airbyte connector that supports the \ @@ -164,6 +166,9 @@ def read( ``"pypi"``, Pathway will prefer the usage of the latest image available on PyPI. Use this option when you need to ensure certain behavior on the local run. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Returns: @@ -337,5 +342,5 @@ def read( schema=_AirbyteRecordSchema, autocommit_duration_ms=max(refresh_interval_ms, 1), name="airbyte", - persistent_id=persistent_id, + unique_name=_get_unique_name(name, kwargs), ) diff --git a/python/pathway/io/bigquery/__init__.py b/python/pathway/io/bigquery/__init__.py index 1f5f7b68..2c1cb4ac 100644 --- a/python/pathway/io/bigquery/__init__.py +++ b/python/pathway/io/bigquery/__init__.py @@ -53,13 +53,18 @@ def _flush_buffer(self) -> None: def write( - table: Table, dataset_name: str, table_name: str, service_user_credentials_file: str + table: Table, + dataset_name: str, + table_name: str, + service_user_credentials_file: str, + *, + name: str | None = None, ) -> None: - """Writes ``table``'s stream of changes into the specified BigQuery table. Please note \ -that the schema of the target table must correspond to the schema of the table that is \ -being outputted and include two additional fields: an integral field ``time``, denoting the \ -ID of the minibatch where the change occurred and an integral field ``diff`` which can be \ -either 1 or -1 and which denotes if the entry was inserted to the table or if it was deleted. + """Writes ``table``'s stream of changes into the specified BigQuery table. Please note + that the schema of the target table must correspond to the schema of the table that is + being outputted and include two additional fields: an integral field ``time``, denoting the + ID of the minibatch where the change occurred and an integral field ``diff`` which can be + either 1 or -1 and which denotes if the entry was inserted to the table or if it was deleted. Note that the modification of the row is denoted with a sequence of two operations: the deletion operation (``diff = -1``) and the insertion operation (``diff = 1``). @@ -68,10 +73,12 @@ def write( table: The table to output. dataset_name: The name of the dataset where the table is located. table_name: The name of the table to be written. - service_user_credentials_file: Google API service user json file. Please \ -follow the instructions provided in the `developer's user guide \ + service_user_credentials_file: Google API service user json file. Please + follow the instructions provided in the `developer's user guide \ `_ \ to obtain them. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -98,5 +105,8 @@ def write( ) output_buffer = _OutputBuffer(dataset_name, table_name, credentials) subscribe( - table, on_change=output_buffer.on_change, on_time_end=output_buffer.on_time_end + table, + on_change=output_buffer.on_change, + on_time_end=output_buffer.on_time_end, + name=name, ) diff --git a/python/pathway/io/csv/__init__.py b/python/pathway/io/csv/__init__.py index a4e444df..0e9126e0 100644 --- a/python/pathway/io/csv/__init__.py +++ b/python/pathway/io/csv/__init__.py @@ -25,7 +25,7 @@ def read( object_pattern: str = "*", with_metadata: bool = False, autocommit_duration_ms: int | None = 1500, - persistent_id: str | None = None, + name: str | None = None, debug_data=None, id_columns: list[str] | None = None, types: dict[str, PathwayType] | None = None, @@ -75,11 +75,9 @@ def read( autocommit_duration_ms: the maximum time between two commits. Every autocommit_duration_ms milliseconds, the updates received by the connector are committed and pushed into Pathway's computation graph. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -170,24 +168,27 @@ def read( csv_settings=csv_settings, autocommit_duration_ms=autocommit_duration_ms, json_field_paths=None, - persistent_id=persistent_id, + name=name, debug_data=debug_data, value_columns=value_columns, primary_key=id_columns, types=types, default_values=default_values, _stacklevel=5, + **kwargs, ) @check_arg_types @trace_user_frame -def write(table: Table, filename: str | PathLike) -> None: +def write(table: Table, filename: str | PathLike, *, name: str | None = None) -> None: """Writes `table`'s stream of updates to a file in delimiter-separated values format. Args: table: Table to be written. filename: Path to the target output file. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -231,4 +232,5 @@ def write(table: Table, filename: str | PathLike) -> None: table, filename=filename, format="csv", + name=name, ) diff --git a/python/pathway/io/debezium/__init__.py b/python/pathway/io/debezium/__init__.py index 9cd7009c..f6b41d76 100644 --- a/python/pathway/io/debezium/__init__.py +++ b/python/pathway/io/debezium/__init__.py @@ -12,7 +12,7 @@ from pathway.internals.table import Table from pathway.internals.table_io import table_from_datasource from pathway.internals.trace import trace_user_frame -from pathway.io._utils import read_schema +from pathway.io._utils import _get_unique_name, read_schema @check_arg_types @@ -25,11 +25,12 @@ def read( schema: type[Schema] | None = None, debug_data=None, autocommit_duration_ms: int | None = 1500, - persistent_id: str | None = None, + name: str | None = None, value_columns: list[str] | None = None, primary_key: list[str] | None = None, types: dict[str, PathwayType] | None = None, default_values: dict[str, Any] | None = None, + **kwargs, ) -> Table: """ Connector, which takes a topic in the format of Debezium @@ -46,11 +47,9 @@ def read( autocommit_duration_ms:the maximum time between two commits. Every autocommit_duration_ms milliseconds, the updates received by the connector are committed and pushed into Pathway's computation graph. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. value_columns: Columns to extract for a table. [will be deprecated soon] primary_key: In case the table should have a primary key generated according to a subset of its columns, the set of columns should be specified in this field. @@ -126,7 +125,6 @@ def read( storage_type="kafka", rdkafka_settings=rdkafka_settings, topic=topic_name, - persistent_id=persistent_id, ) schema, data_format_definition = read_schema( schema=schema, @@ -136,7 +134,8 @@ def read( default_values=default_values, ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs), ) data_format = api.DataFormat( format_type="debezium", debezium_db_type=db_type, **data_format_definition diff --git a/python/pathway/io/deltalake/__init__.py b/python/pathway/io/deltalake/__init__.py index 3e0fbdcb..b90f169d 100644 --- a/python/pathway/io/deltalake/__init__.py +++ b/python/pathway/io/deltalake/__init__.py @@ -17,7 +17,7 @@ from pathway.internals.table import Table from pathway.internals.table_io import table_from_datasource from pathway.internals.trace import trace_user_frame -from pathway.io._utils import internal_connector_mode, read_schema +from pathway.io._utils import _get_unique_name, internal_connector_mode, read_schema from pathway.io.minio import MinIOSettings from pathway.io.s3 import DigitalOceanS3Settings, WasabiS3Settings @@ -44,8 +44,9 @@ def read( AwsS3Settings | MinIOSettings | WasabiS3Settings | DigitalOceanS3Settings | None ) = None, autocommit_duration_ms: int | None = 1500, - persistent_id: str | None = None, + name: str | None = None, debug_data: Any = None, + **kwargs, ) -> Table: """ Reads a table from Delta Lake. Currently, local and S3 lakes are supported. The table @@ -69,11 +70,9 @@ def read( endpoint, which is necessary for buckets hosted outside of Amazon AWS. If the custom endpoint is left blank, the authorized user's credentials for S3 will be used. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. autocommit_duration_ms: The maximum time between two commits. Every ``autocommit_duration_ms`` milliseconds, the updates received by the connector are committed and pushed into Pathway's computation graph. @@ -151,7 +150,6 @@ def read( aws_s3_settings=_engine_s3_connection_settings( uri, prepared_connection_settings ), - persistent_id=persistent_id, ) data_format = api.DataFormat( format_type="transparent", @@ -159,7 +157,8 @@ def read( ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs), ) return table_from_datasource( datasource.GenericDataSource( @@ -184,6 +183,7 @@ def write( AwsS3Settings | MinIOSettings | WasabiS3Settings | DigitalOceanS3Settings | None ) = None, min_commit_frequency: int | None = 60_000, + name: str | None = None, ) -> None: """ Writes the stream of changes from ``table`` into `Delta Lake _` data @@ -217,6 +217,8 @@ def write( or \ `optimize `_ operations afterwards. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -276,6 +278,7 @@ def write( data_storage, data_format, datasink_name="deltalake", + unique_name=name, ) ) diff --git a/python/pathway/io/elasticsearch/__init__.py b/python/pathway/io/elasticsearch/__init__.py index 7a41550c..43233c3a 100644 --- a/python/pathway/io/elasticsearch/__init__.py +++ b/python/pathway/io/elasticsearch/__init__.py @@ -49,7 +49,14 @@ def engine_es_auth(self) -> api.ElasticSearchAuth: @check_arg_types @trace_user_frame -def write(table: Table, host: str, auth: ElasticSearchAuth, index_name: str) -> None: +def write( + table: Table, + host: str, + auth: ElasticSearchAuth, + index_name: str, + *, + name: str | None = None, +) -> None: """Write a table to a given index in ElasticSearch. Args: @@ -57,6 +64,8 @@ def write(table: Table, host: str, auth: ElasticSearchAuth, index_name: str) -> host: the host and port, on which Elasticsearch server works. auth: credentials for Elasticsearch authorization. index_name: name of the index, which gets the docs. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -113,5 +122,6 @@ def write(table: Table, host: str, auth: ElasticSearchAuth, index_name: str) -> data_storage, data_format, datasink_name="elasticsearch", + unique_name=name, ) ) diff --git a/python/pathway/io/fs/__init__.py b/python/pathway/io/fs/__init__.py index 24b640e4..b6bf2ef3 100644 --- a/python/pathway/io/fs/__init__.py +++ b/python/pathway/io/fs/__init__.py @@ -15,6 +15,7 @@ from pathway.internals.trace import trace_user_frame from pathway.io._utils import ( CsvParserSettings, + _get_unique_name, construct_schema_and_data_format, internal_connector_mode, internal_read_method, @@ -38,7 +39,7 @@ def read( json_field_paths: dict[str, str] | None = None, object_pattern: str = "*", with_metadata: bool = False, - persistent_id: str | None = None, + name: str | None = None, autocommit_duration_ms: int | None = 1500, debug_data: Any = None, value_columns: list[str] | None = None, @@ -46,6 +47,7 @@ def read( types: dict[str, PathwayType] | None = None, default_values: dict[str, Any] | None = None, _stacklevel: int = 1, + **kwargs, ) -> Table: """Reads a table from one or several files with the specified format. @@ -90,11 +92,9 @@ def read( (3) seen_at is a UNIX timestamp of when they file was found by the engine; (4) owner - Name of the file owner (only for Un); (5) path - Full file path of the source row. (6) size - File size in bytes. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. value_columns: Names of the columns to be extracted from the files. [will be deprecated soon] primary_key: In case the table should have a primary key generated according to @@ -234,7 +234,6 @@ def read( csv_parser_settings=csv_settings.api_settings if csv_settings else None, mode=internal_connector_mode(mode), object_pattern=object_pattern, - persistent_id=persistent_id, ) else: data_storage = api.DataStorage( @@ -243,7 +242,6 @@ def read( mode=internal_connector_mode(mode), read_method=internal_read_method(format), object_pattern=object_pattern, - persistent_id=persistent_id, ) schema, data_format = construct_schema_and_data_format( @@ -260,7 +258,8 @@ def read( ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs, _stacklevel + 5), ) return table_from_datasource( datasource.GenericDataSource( @@ -276,7 +275,13 @@ def read( @check_arg_types @trace_user_frame -def write(table: Table, filename: str | PathLike, format: str) -> None: +def write( + table: Table, + filename: str | PathLike, + format: str, + *, + name: str | None = None, +) -> None: """Writes ``table``'s stream of updates to a file in the given format. Args: @@ -284,6 +289,8 @@ def write(table: Table, filename: str | PathLike, format: str) -> None: filename: Path to the target output file. format: Format to use for data output. Currently, there are two supported formats: "json" and "csv". + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -364,4 +371,8 @@ def write(table: Table, filename: str | PathLike, format: str) -> None: value_fields=_format_output_value_fields(table), ) - table.to(datasink.GenericDataSink(data_storage, data_format, datasink_name="fs")) + table.to( + datasink.GenericDataSink( + data_storage, data_format, datasink_name="fs", unique_name=name + ) + ) diff --git a/python/pathway/io/gdrive/__init__.py b/python/pathway/io/gdrive/__init__.py index 1640e958..d54caab6 100644 --- a/python/pathway/io/gdrive/__init__.py +++ b/python/pathway/io/gdrive/__init__.py @@ -278,7 +278,7 @@ def __init__( object_size_limit: int | None, file_name_pattern: list | str | None, ) -> None: - super().__init__() + super().__init__(datasource_name="gdrive") self._credentials_factory = credentials_factory self._refresh_interval = refresh_interval self._root = root @@ -346,6 +346,8 @@ def read( service_user_credentials_file: str, with_metadata: bool = False, file_name_pattern: list | str | None = None, + name: str | None = None, + **kwargs, ) -> pw.Table: """Reads a table from a Google Drive directory or file. @@ -371,6 +373,10 @@ def read( file_name_pattern: glob pattern (or list of patterns) to be used to filter files based on their names. Defaults to `None` which doesn't filter anything. Doesn't apply to folder names. For example, `*.pdf` will only return files that has `.pdf` extension. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. + Returns: The table read. @@ -402,4 +408,10 @@ def credentials_factory() -> ServiceCredentials: file_name_pattern=file_name_pattern, ) - return pw.io.python.read(subject, format="binary", name="gdrive") + return pw.io.python.read( + subject, + format="binary", + name=name, + _stacklevel=4, + **kwargs, + ) diff --git a/python/pathway/io/http/__init__.py b/python/pathway/io/http/__init__.py index cc3b90ff..d75d0523 100644 --- a/python/pathway/io/http/__init__.py +++ b/python/pathway/io/http/__init__.py @@ -47,6 +47,7 @@ def read( primary_key: list[str] | None = None, types: dict[str, PathwayType] | None = None, default_values: dict[str, Any] | None = None, + name: str | None = None, ): """Reads a table from an HTTP stream. @@ -88,6 +89,9 @@ def read( default_values: dictionary containing default values for columns replacing blank entries. The default value of the column must be specified explicitly, otherwise there will be no default value. [will be deprecated soon] + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Examples: @@ -149,7 +153,8 @@ def read( default_values=default_values, autocommit_duration_ms=autocommit_duration_ms, debug_data=debug_data, - name="http", + name=name, + _stacklevel=5, ) @@ -170,6 +175,7 @@ def write( headers: dict[str, str] | None = None, allow_redirects: bool = True, retry_codes: tuple | None = (429, 500, 502, 503, 504), + name: str | None = None, ) -> None: """Sends the stream of updates from the table to the specified HTTP API. @@ -195,6 +201,8 @@ def write( JSON, it will be defaulted to "application/json". headers: request headers in the form of dict. Wildcards are allowed both, in keys and in values. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Wildcards: @@ -288,7 +296,7 @@ def callback(key: Pointer, row: dict[str, Any], time: int, is_addition: bool): data=payload, ) - subscribe(table, callback) + subscribe(table, callback, name=name) __all__ = [ diff --git a/python/pathway/io/http/_server.py b/python/pathway/io/http/_server.py index 298ba46c..20d76bbc 100644 --- a/python/pathway/io/http/_server.py +++ b/python/pathway/io/http/_server.py @@ -99,14 +99,14 @@ def add_example(self, id, summary, values): Adds an example to the collection. Args: - id: Short and unique ID for the example. It is used for naming the example \ -within the Open API schema. By using `default` as an ID, you can set the example \ -default for the readers, while users will be able to select another ones via the \ -dropdown menu. - summary: Human-readable summary of the example, describing what is shown. \ -It is shown in the automatically generated dropdown menu. - values: The key-value dictionary, a mapping from the fields described in \ -schema to their values in the example. + id: Short and unique ID for the example. It is used for naming the example + within the Open API schema. By using `default` as an ID, you can set the example + default for the readers, while users will be able to select another ones via the + dropdown menu. + summary: Human-readable summary of the example, describing what is shown. + It is shown in the automatically generated dropdown menu. + values: The key-value dictionary, a mapping from the fields described in + schema to their values in the example. Returns: None @@ -131,8 +131,8 @@ class EndpointDocumentation: summary: Short endpoint description shown as a hint in the endpoints list. description: Comprehensive description for the endpoint. tags: Tags for grouping the endpoints. - method_types: If set, Pathway will document only the given method types. This \ -way, one can exclude certain endpoints and methods from being documented. + method_types: If set, Pathway will document only the given method types. This + way, one can exclude certain endpoints and methods from being documented. """ DEFAULT_RESPONSES_DESCRIPTION = { @@ -336,11 +336,11 @@ class PathwayWebserver: Args: host: TCP/IP host or a sequence of hosts for the created endpoint. port: Port for the created endpoint. - with_schema_endpoint: If set to True, the server will also provide ``/_schema`` \ -endpoint containing Open API 3.0.3 schema for the handlers generated with \ -``pw.io.http.rest_connector`` calls. - with_cors: If set to True, the server will allow cross-origin requests on the \ -added endpoints. + with_schema_endpoint: If set to True, the server will also provide ``/_schema`` + endpoint containing Open API 3.0.3 schema for the handlers generated with + ``pw.io.http.rest_connector`` calls. + with_cors: If set to True, the server will allow cross-origin requests on the + added endpoints. """ _host: str @@ -505,7 +505,7 @@ def __init__( documentation: EndpointDocumentation = EndpointDocumentation(), cache_strategy: CacheStrategy | None = None, ) -> None: - super().__init__() + super().__init__(datasource_name="rest-connector") self._webserver = webserver self._tasks = webserver._tasks self._schema = schema @@ -645,8 +645,8 @@ def rest_connector( respective request's responses. Args: - webserver: configuration object containing host and port information. You only \ -need to create only one instance of this class per single host-port pair; + webserver: configuration object containing host and port information. You only + need to create only one instance of this class per single host-port pair; route: route which will be listened to by the web server; schema: schema of the resulting table; methods: HTTP methods that this endpoint will accept; @@ -664,16 +664,16 @@ def rest_connector( Returns: table: the table read; - response_writer: a callable, where the result table should be provided. The \ -result table must contain columns `query_id` corresponding to the primary key of an \ -object from the input table and `result`, corresponding to the endpoint's return value. + response_writer: a callable, where the result table should be provided. The + result table must contain columns `query_id` corresponding to the primary key of an + object from the input table and `result`, corresponding to the endpoint's return value. Example: - Let's consider the following example: there is a collection of words that are \ -received through HTTP REST endpoint `/uppercase` located at `127.0.0.1`, port `9999`. \ -The Pathway program processes this table by converting these words to the upper case. \ -This conversion result must be provided to the user on the output. + Let's consider the following example: there is a collection of words that are + received through HTTP REST endpoint `/uppercase` located at `127.0.0.1`, port `9999`. + The Pathway program processes this table by converting these words to the upper case. + This conversion result must be provided to the user on the output. Then, you can proceed with the following REST connector configuration code. @@ -703,11 +703,11 @@ def rest_connector( ... ) >>> response_writer(uppercase_words) - Please note that you don't need to create another web server object if you need to \ -have more than one endpoint running on the same host and port. For example, if you need \ -to create another endpoint that converts words to lower case, in the same way, you \ -need to reuse the existing `webserver` object. That is, the configuration would start \ -with: + Please note that you don't need to create another web server object if you need to + have more than one endpoint running on the same host and port. For example, if you need + to create another endpoint that converts words to lower case, in the same way, you + need to reuse the existing `webserver` object. That is, the configuration would start + with: >>> words_for_lowercase, response_writer_for_lowercase = pw.io.http.rest_connector( ... webserver=webserver, @@ -772,7 +772,6 @@ def rest_connector( schema=schema, format="json", autocommit_duration_ms=autocommit_duration_ms, - name="rest-connector", ) def response_writer(responses: pw.Table): diff --git a/python/pathway/io/http/_streaming.py b/python/pathway/io/http/_streaming.py index 38cf9798..daae5346 100644 --- a/python/pathway/io/http/_streaming.py +++ b/python/pathway/io/http/_streaming.py @@ -27,7 +27,7 @@ def __init__( delimiter: str | bytes | None = None, response_mapper: Callable[[str | bytes], bytes] | None = None, ) -> None: - super().__init__() + super().__init__(datasource_name="http") self._url = url self._headers = headers self._sender = sender diff --git a/python/pathway/io/iceberg/__init__.py b/python/pathway/io/iceberg/__init__.py index 1f67255c..a4fda523 100644 --- a/python/pathway/io/iceberg/__init__.py +++ b/python/pathway/io/iceberg/__init__.py @@ -10,7 +10,7 @@ from pathway.internals.table import Table from pathway.internals.table_io import table_from_datasource from pathway.internals.trace import trace_user_frame -from pathway.io._utils import internal_connector_mode, read_schema +from pathway.io._utils import _get_unique_name, internal_connector_mode, read_schema @check_arg_types @@ -24,8 +24,9 @@ def read( mode: str = "streaming", warehouse: str | None = None, autocommit_duration_ms: int | None = 1500, - persistent_id: str | None = None, + name: str | None = None, debug_data: Any = None, + **kwargs, ) -> Table: """ Reads a table from Apache Iceberg. If ran in a streaming mode, the connector tracks @@ -50,11 +51,9 @@ def read( autocommit_duration_ms: The maximum time between two commits. Every ``autocommit_duration_ms`` milliseconds, the updates received by the connector are committed and pushed into Pathway's computation graph. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -111,7 +110,6 @@ def read( table_name=table_name, namespace=namespace, mode=internal_connector_mode(mode), - persistent_id=persistent_id, ) data_format = api.DataFormat( format_type="transparent", @@ -119,7 +117,8 @@ def read( ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs), ) return table_from_datasource( datasource.GenericDataSource( @@ -144,6 +143,7 @@ def write( *, warehouse: str | None = None, min_commit_frequency: int | None = 60_000, + name: str | None = None, ): """ Writes the stream of changes from ``table`` into `Iceberg `_ @@ -170,6 +170,8 @@ def write( commit in Iceberg creates a new Parquet file and writes an entry in the transaction log. Therefore, it is advisable to limit the frequency of commits to reduce the overhead of processing the resulting table. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -232,5 +234,6 @@ def write( data_storage, data_format, datasink_name="iceberg", + unique_name=name, ) ) diff --git a/python/pathway/io/jsonlines/__init__.py b/python/pathway/io/jsonlines/__init__.py index 73be3a2e..db166cb6 100644 --- a/python/pathway/io/jsonlines/__init__.py +++ b/python/pathway/io/jsonlines/__init__.py @@ -24,12 +24,13 @@ def read( object_pattern: str = "*", with_metadata: bool = False, autocommit_duration_ms: int | None = 1500, - persistent_id: str | None = None, + name: str | None = None, debug_data=None, value_columns: list[str] | None = None, primary_key: list[str] | None = None, types: dict[str, PathwayType] | None = None, default_values: dict[str, Any] | None = None, + **kwargs, ) -> Table: """Reads a table from one or several files in jsonlines format. @@ -39,40 +40,38 @@ def read( the modification time. Args: - path: Path to the file or to the folder with files or \ -`glob `_ pattern for the \ -objects to be read. The connector will read the contents of all matching files as well \ -as recursively read the contents of all matching folders. + path: Path to the file or to the folder with files or + `glob `_ pattern for the + objects to be read. The connector will read the contents of all matching files as well + as recursively read the contents of all matching folders. schema: Schema of the resulting table. - mode: Denotes how the engine polls the new data from the source. Currently \ -"streaming" and "static" are supported. If set to "streaming" the engine will wait for \ -the updates in the specified directory. It will track file additions, deletions, and \ -modifications and reflect these events in the state. For example, if a file was deleted,\ -"streaming" mode will also remove rows obtained by reading this file from the table. On \ -the other hand, the "static" mode will only consider the available data and ingest all \ -of it in one commit. The default value is "streaming". + mode: Denotes how the engine polls the new data from the source. Currently + "streaming" and "static" are supported. If set to "streaming" the engine will wait for + the updates in the specified directory. It will track file additions, deletions, and + modifications and reflect these events in the state. For example, if a file was deleted, + "streaming" mode will also remove rows obtained by reading this file from the table. On + the other hand, the "static" mode will only consider the available data and ingest all + of it in one commit. The default value is "streaming". json_field_paths: This field allows to map field names into path in the field. For the field which require such mapping, it should be given in the format ``: ``, where the path to be mapped needs to be a `JSON Pointer (RFC 6901) `_. - object_pattern: Unix shell style pattern for filtering only certain files in the \ -directory. Ignored in case a path to a single file is specified. This value will be \ -deprecated soon, please use glob pattern in ``path`` instead. - with_metadata: When set to true, the connector will add an additional column \ -named ``_metadata`` to the table. This column will be a JSON field that will contain two \ -optional fields - ``created_at`` and ``modified_at``. These fields will have integral \ -UNIX timestamps for the creation and modification time respectively. Additionally, the \ -column will also have an optional field named ``owner`` that will contain the name of \ -the file owner (applicable only for Un). Finally, the column will also contain a field \ -named ``path`` that will show the full path to the file from where a row was filled. + object_pattern: Unix shell style pattern for filtering only certain files in the + directory. Ignored in case a path to a single file is specified. This value will be + deprecated soon, please use glob pattern in ``path`` instead. + with_metadata: When set to true, the connector will add an additional column + named ``_metadata`` to the table. This column will be a JSON field that will contain two + optional fields - ``created_at`` and ``modified_at``. These fields will have integral + UNIX timestamps for the creation and modification time respectively. Additionally, the + column will also have an optional field named ``owner`` that will contain the name of + the file owner (applicable only for Un). Finally, the column will also contain a field + named ``path`` that will show the full path to the file from where a row was filled. autocommit_duration_ms: the maximum time between two commits. Every - autocommit_duration_ms milliseconds, the updates received by the connector are - committed and pushed into Pathway's computation graph. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + autocommit_duration_ms milliseconds, the updates received by the connector are + committed and pushed into Pathway's computation graph. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. value_columns: Names of the columns to be extracted from the files. [will be deprecated soon] primary_key: In case the table should have a primary key generated according to @@ -172,7 +171,7 @@ def read( mode=mode, json_field_paths=json_field_paths, debug_data=debug_data, - persistent_id=persistent_id, + name=name, autocommit_duration_ms=autocommit_duration_ms, value_columns=value_columns, object_pattern=object_pattern, @@ -181,17 +180,20 @@ def read( types=types, default_values=default_values, _stacklevel=5, + **kwargs, ) @check_arg_types @trace_user_frame -def write(table: Table, filename: str | PathLike) -> None: +def write(table: Table, filename: str | PathLike, *, name: str | None = None) -> None: """Writes ``table``'s stream of updates to a file in jsonlines format. Args: table: Table to be written. filename: Path to the target output file. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -230,4 +232,4 @@ def write(table: Table, filename: str | PathLike) -> None: you have read three rows and all of them were added to the collection (``diff = 1``). """ - pw.io.fs.write(table, filename=filename, format="json") + pw.io.fs.write(table, filename=filename, format="json", name=name) diff --git a/python/pathway/io/kafka/__init__.py b/python/pathway/io/kafka/__init__.py index 5157dcfa..c2edf52a 100644 --- a/python/pathway/io/kafka/__init__.py +++ b/python/pathway/io/kafka/__init__.py @@ -16,6 +16,7 @@ from pathway.internals.trace import trace_user_frame from pathway.io._utils import ( MessageQueueOutputFormat, + _get_unique_name, check_deprecated_kwargs, check_raw_and_plaintext_only_kwargs_for_message_queues, construct_schema_and_data_format, @@ -37,7 +38,7 @@ def read( with_metadata: bool = False, start_from_timestamp_ms: int | None = None, parallel_readers: int | None = None, - persistent_id: str | None = None, + name: str | None = None, value_columns: list[str] | None = None, primary_key: list[str] | None = None, types: dict[str, PathwayType] | None = None, @@ -89,11 +90,9 @@ def read( will be taken. This number also can't be greater than the number of Pathway engine threads, and will be reduced to the number of engine threads, if it exceeds. - persistent_id: (unstable) An identifier, under which the state of the table will - be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. value_columns: Columns to extract for a table, required for format other than "raw". [will be deprecated soon] primary_key: In case the table should have a primary key generated according to @@ -133,8 +132,8 @@ def read( ... "sasl.password": os.environ["KAFKA_PASSWORD"] ... } - To connect to the topic "animals" and accept messages, the connector must be used \ - as follows, depending on the format: + To connect to the topic "animals" and accept messages, the connector must be used + as follows, depending on the format: Raw version: @@ -262,7 +261,6 @@ def read( rdkafka_settings=rdkafka_settings, topic=topic, parallel_readers=parallel_readers, - persistent_id=persistent_id, start_from_timestamp_ms=start_from_timestamp_ms, mode=api.ConnectorMode.STREAMING, ) @@ -280,7 +278,8 @@ def read( _stacklevel=5, ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs, stacklevel=_stacklevel + 5), ) return table_from_datasource( datasource.GenericDataSource( @@ -307,7 +306,8 @@ def simple_read( autocommit_duration_ms: int | None = 1500, json_field_paths: dict[str, str] | None = None, parallel_readers: int | None = None, - persistent_id: str | None = None, + name: str | None = None, + **kwargs, ) -> Table: """Simplified method to read data from Kafka. Only requires the server address and the topic name. If you have any kind of authentication or require fine-tuning of the @@ -330,9 +330,9 @@ def simple_read( Args: server: Address of the server. topic: Name of topic in Kafka from which the data should be read. - read_only_new: If set to `True` only the entries which appear after the start \ -of the program will be read. Otherwise, the read will be done from the beginning of the\ -topic. + read_only_new: If set to `True` only the entries which appear after the start + of the program will be read. Otherwise, the read will be done from the + beginning of the topic. schema: Schema of the resulting table. format: format of the input data, "raw", "plaintext", or "json". debug_data: Static data replacing original one when debug mode is active. @@ -349,11 +349,9 @@ def simple_read( will be taken. This number also can't be greater than the number of Pathway engine threads, and will be reduced to the number of engine threads, if it exceeds. - persistent_id: (unstable) An identifier, under which the state of the table will - be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Returns: Table: The table read. @@ -387,10 +385,13 @@ def simple_read( autocommit_duration_ms=autocommit_duration_ms, json_field_paths=json_field_paths, parallel_readers=parallel_readers, - persistent_id=persistent_id, + name=name, + _stacklevel=5, + **kwargs, ) +# TODO: Remove on March 11, 2025 @check_arg_types @trace_user_frame def read_from_upstash( @@ -406,7 +407,8 @@ def read_from_upstash( autocommit_duration_ms: int | None = 1500, json_field_paths: dict[str, str] | None = None, parallel_readers: int | None = None, - persistent_id: str | None = None, + name: str | None = None, + **kwargs, ) -> Table: """Simplified method to read data from Kafka instance hosted in Upstash. It requires endpoint address and topic along with credentials. @@ -426,15 +428,15 @@ def read_from_upstash( taken from the respective parsed JSON fields. Args: - endpoint: Upstash endpoint for the sought queue, which can be found on \ -"Details" page. + endpoint: Upstash endpoint for the sought queue, which can be found on + "Details" page. username: Username generated for this queue. - password: Password generated for this queue. These credentials are also \ -available on "Details" page. + password: Password generated for this queue. These credentials are also + available on "Details" page. topic: Name of topic in Kafka from which the data should be read. - read_only_new: If set to `True` only the entries which appear after the start \ -of the program will be read. Otherwise, the read will be done from the beginning of the\ -topic. + read_only_new: If set to `True` only the entries which appear after the start + of the program will be read. Otherwise, the read will be done from the + beginning of the topic. schema: Schema of the resulting table. format: format of the input data, "raw", "plaintext", or "json". debug_data: Static data replacing original one when debug mode is active. @@ -451,11 +453,9 @@ def read_from_upstash( will be taken. This number also can't be greater than the number of Pathway engine threads, and will be reduced to the number of engine threads, if it exceeds. - persistent_id: (unstable) An identifier, under which the state of the table will - be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Returns: Table: The table read. @@ -500,7 +500,9 @@ def read_from_upstash( autocommit_duration_ms=autocommit_duration_ms, json_field_paths=json_field_paths, parallel_readers=parallel_readers, - persistent_id=persistent_id, + name=name, + _stacklevel=5, + **kwargs, ) @@ -517,6 +519,7 @@ def write( key: ColumnReference | None = None, value: ColumnReference | None = None, headers: Iterable[ColumnReference] | None = None, + name: str | None = None, ) -> None: """Write a table to a given topic on a Kafka instance. @@ -563,7 +566,8 @@ def write( headers. These headers are named in the same way as fields that are forwarded and correspond to the string representations of the respective values encoded in UTF-8. If a binary column is requested, it will be produced "as is" in the respective header. - + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -682,6 +686,9 @@ def write( table.to( datasink.GenericDataSink( - data_storage, output_format.data_format, datasink_name="kafka" + data_storage, + output_format.data_format, + datasink_name="kafka", + unique_name=name, ) ) diff --git a/python/pathway/io/logstash/__init__.py b/python/pathway/io/logstash/__init__.py index 0dbffc58..d9980fba 100644 --- a/python/pathway/io/logstash/__init__.py +++ b/python/pathway/io/logstash/__init__.py @@ -18,9 +18,11 @@ def write( retry_policy: RetryPolicy = RetryPolicy.default(), connect_timeout_ms: int | None = None, request_timeout_ms: int | None = None, + *, + name: str | None = None, ) -> None: - """Sends the stream of updates from the table to \ -`HTTP input ` + """Sends the stream of updates from the table to + `HTTP input ` of Logstash. The data is sent in the format of flat JSON objects, with two extra fields for time and diff. @@ -29,10 +31,12 @@ def write( endpoint: Logstash endpoint, accepting entries; n_retries: number of retries in case of failure; retry_policy: policy of delays or backoffs for the retries; - connect_timeout_ms: connection timeout, specified in milliseconds. In case \ -it's None, no restrictions on connection duration will be applied; - request_timeout_ms: request timeout, specified in milliseconds. In case it's \ -None, no restrictions on request duration will be applied. + connect_timeout_ms: connection timeout, specified in milliseconds. In cas it's None, + no restrictions on connection duration will be applied; + request_timeout_ms: request timeout, specified in milliseconds. In case it's None, + no restrictions on request duration will be applied. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Example: @@ -67,4 +71,5 @@ def write( retry_policy=retry_policy, connect_timeout_ms=connect_timeout_ms, request_timeout_ms=request_timeout_ms, + name=name, ) diff --git a/python/pathway/io/minio/__init__.py b/python/pathway/io/minio/__init__.py index 7a7330af..fec0d073 100644 --- a/python/pathway/io/minio/__init__.py +++ b/python/pathway/io/minio/__init__.py @@ -21,9 +21,9 @@ class MinIOSettings: access_key: Access key for the bucket. secret_access_key: Secret access key for the bucket. region: Region of the bucket. - with_path_style: Whether to use path-style addresses for bucket access. It defaults \ -to True as this is the most widespread way to access MinIO, but can be overridden in case \ -of a custom configuration. + with_path_style: Whether to use path-style addresses for bucket access. It defaults + to True as this is the most widespread way to access MinIO, but can be overridden + in case of a custom configuration. """ def __init__( @@ -67,9 +67,10 @@ def read( csv_settings: CsvParserSettings | None = None, json_field_paths: dict[str, str] | None = None, downloader_threads_count: int | None = None, - persistent_id: str | None = None, + name: str | None = None, autocommit_duration_ms: int | None = 1500, debug_data: Any = None, + **kwargs, ) -> Table: """Reads a table from one or several objects from S3 bucket in MinIO. @@ -110,11 +111,9 @@ def read( of the bucket under the given path. It defaults to the number of cores available on the machine. It is recommended to increase the number of threads if your bucket contains many small files. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -158,8 +157,10 @@ def read( mode=mode, with_metadata=with_metadata, autocommit_duration_ms=autocommit_duration_ms, - persistent_id=persistent_id, + name=name, json_field_paths=json_field_paths, downloader_threads_count=downloader_threads_count, debug_data=debug_data, + _stacklevel=5, + **kwargs, ) diff --git a/python/pathway/io/mongodb/__init__.py b/python/pathway/io/mongodb/__init__.py index c3b9d836..2d124bea 100644 --- a/python/pathway/io/mongodb/__init__.py +++ b/python/pathway/io/mongodb/__init__.py @@ -18,6 +18,7 @@ def write( database: str, collection: str, max_batch_size: int | None = None, + name: str | None = None, ) -> None: """Writes ``table``'s stream of updates to a MongoDB table. @@ -29,8 +30,8 @@ def write( and ``diff`` shows the nature of the change: ``1`` means a row was added and ``-1`` means a row was deleted. - **Note:** Since MongoDB \ -`stores DateTime in milliseconds `_,\ + **Note:** Since MongoDB + `stores DateTime in milliseconds `_, the `Duration `_ type is also serialized as an integer number of milliseconds for consistency. @@ -41,6 +42,8 @@ def write( database: The name of the database to update. collection: The name of the collection to write to. max_batch_size: The maximum number of entries to insert in one batch. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -185,6 +188,9 @@ def write( table.to( datasink.GenericDataSink( - data_storage, data_format, datasink_name="mongodb.sink" + data_storage, + data_format, + datasink_name="mongodb.sink", + unique_name=name, ) ) diff --git a/python/pathway/io/nats/__init__.py b/python/pathway/io/nats/__init__.py index 0abd43a0..19b8e21f 100644 --- a/python/pathway/io/nats/__init__.py +++ b/python/pathway/io/nats/__init__.py @@ -13,6 +13,7 @@ from pathway.internals.trace import trace_user_frame from pathway.io._utils import ( MessageQueueOutputFormat, + _get_unique_name, check_raw_and_plaintext_only_kwargs_for_message_queues, construct_schema_and_data_format, ) @@ -29,8 +30,9 @@ def read( autocommit_duration_ms: int | None = 1500, json_field_paths: dict[str, str] | None = None, parallel_readers: int | None = None, - persistent_id: str | None = None, + name: str | None = None, debug_data=None, + **kwargs, ) -> Table: """Reads data from a specified NATS topic. @@ -64,11 +66,9 @@ def read( specified, it defaults to ``min(pathway_threads, total_partitions)``. It can't exceed the number of Pathway engine threads and will be reduced if necessary. - persistent_id: (unstable) An identifier, under which the state of the table will - be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -124,7 +124,6 @@ def read( path=uri, topic=topic, parallel_readers=parallel_readers, - persistent_id=persistent_id, mode=api.ConnectorMode.STREAMING, ) schema, data_format = construct_schema_and_data_format( @@ -134,7 +133,8 @@ def read( json_field_paths=json_field_paths, ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs), ) return table_from_datasource( datasource.GenericDataSource( @@ -160,6 +160,7 @@ def write( delimiter: str = ",", value: ColumnReference | None = None, headers: Iterable[ColumnReference] | None = None, + name: str | None = None, ) -> None: """Writes data into the specified NATS topic. @@ -200,6 +201,8 @@ def write( to the string representations of the respective values encoded in UTF-8. Note that due to NATS constraints imposed on headers, the binary fields must also be UTF-8 serializable. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Example: @@ -272,6 +275,9 @@ def write( table.to( datasink.GenericDataSink( - data_storage, output_format.data_format, datasink_name="nats" + data_storage, + output_format.data_format, + datasink_name="nats", + unique_name=name, ) ) diff --git a/python/pathway/io/null/__init__.py b/python/pathway/io/null/__init__.py index ada572a3..1aab4fcd 100644 --- a/python/pathway/io/null/__init__.py +++ b/python/pathway/io/null/__init__.py @@ -10,7 +10,7 @@ @check_arg_types @trace_user_frame -def write(table: Table) -> None: +def write(table: Table, *, name: str | None = None) -> None: """Writes ``table``'s stream of updates to the empty sink. Inside this routine, the data is formatted into the empty object, and then doesn't @@ -18,6 +18,8 @@ def write(table: Table) -> None: Args: table: Table to be written. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -45,5 +47,6 @@ def write(table: Table) -> None: data_storage, data_format, datasink_name="null", + unique_name=name, ) ) diff --git a/python/pathway/io/plaintext/__init__.py b/python/pathway/io/plaintext/__init__.py index 4d3ae2de..3f50b054 100644 --- a/python/pathway/io/plaintext/__init__.py +++ b/python/pathway/io/plaintext/__init__.py @@ -18,9 +18,10 @@ def read( mode: str = "streaming", object_pattern: str = "*", with_metadata: bool = False, - persistent_id: str | None = None, + name: str | None = None, autocommit_duration_ms: int | None = 1500, debug_data=None, + **kwargs, ) -> Table: """Reads a table from a text file or a directory of text files. The resulting table will consist of a single column ``data``, and have the number of rows equal to the number @@ -31,35 +32,33 @@ def read( modification time is, the earlier the file will be passed to the engine. Args: - path: Path to the file or to the folder with files or \ -`glob `_ pattern for the \ -objects to be read. The connector will read the contents of all matching files as well \ -as recursively read the contents of all matching folders. - mode: Denotes how the engine polls the new data from the source. Currently \ -"streaming" and "static" are supported. If set to "streaming" the engine will wait for \ -the updates in the specified directory. It will track file additions, deletions, and \ -modifications and reflect these events in the state. For example, if a file was deleted,\ -"streaming" mode will also remove rows obtained by reading this file from the table. On \ -the other hand, the "static" mode will only consider the available data and ingest all \ -of it in one commit. The default value is "streaming". - object_pattern: Unix shell style pattern for filtering only certain files in the \ -directory. Ignored in case a path to a single file is specified. This value will be \ -deprecated soon, please use glob pattern in ``path`` instead. - with_metadata: When set to true, the connector will add an additional column \ -named ``_metadata`` to the table. This column will be a JSON field that will contain two \ -optional fields - ``created_at`` and ``modified_at``. These fields will have integral \ -UNIX timestamps for the creation and modification time respectively. Additionally, the \ -column will also have an optional field named ``owner`` that will contain the name of \ -the file owner (applicable only for Un). Finally, the column will also contain a field \ -named ``path`` that will show the full path to the file from where a row was filled. - persistent_id: (unstable) An identifier, under which the state of the table \ -will be persisted or ``None``, if there is no need to persist the state of this table. \ -When a program restarts, it restores the state for all input tables according to what \ -was saved for their ``persistent_id``. This way it's possible to configure the start of \ -computations from the moment they were terminated last time. + path: Path to the file or to the folder with files or + `glob `_ pattern for the + objects to be read. The connector will read the contents of all matching files as well + as recursively read the contents of all matching folders. + mode: Denotes how the engine polls the new data from the source. Currently + "streaming" and "static" are supported. If set to "streaming" the engine will wait for + the updates in the specified directory. It will track file additions, deletions, and + modifications and reflect these events in the state. For example, if a file was deleted, + "streaming" mode will also remove rows obtained by reading this file from the table. On + the other hand, the "static" mode will only consider the available data and ingest all + of it in one commit. The default value is "streaming". + object_pattern: Unix shell style pattern for filtering only certain files in the + directory. Ignored in case a path to a single file is specified. This value will be + deprecated soon, please use glob pattern in ``path`` instead. + with_metadata: When set to true, the connector will add an additional column + named ``_metadata`` to the table. This column will be a JSON field that will contain two + optional fields - ``created_at`` and ``modified_at``. These fields will have integral + UNIX timestamps for the creation and modification time respectively. Additionally, the + column will also have an optional field named ``owner`` that will contain the name of + the file owner (applicable only for Un). Finally, the column will also contain a field + named ``path`` that will show the full path to the file from where a row was filled. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. autocommit_duration_ms: the maximum time between two commits. Every - autocommit_duration_ms milliseconds, the updates received by the connector are - committed and pushed into Pathway's computation graph. + ``autocommit_duration_ms`` milliseconds, the updates received by the connector are + committed and pushed into Pathway's computation graph. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -77,8 +76,9 @@ def read( mode=mode, object_pattern=object_pattern, with_metadata=with_metadata, - persistent_id=persistent_id, + name=name, autocommit_duration_ms=autocommit_duration_ms, debug_data=debug_data, _stacklevel=5, + **kwargs, ) diff --git a/python/pathway/io/postgres/__init__.py b/python/pathway/io/postgres/__init__.py index 201fb3fe..31dfe5e5 100644 --- a/python/pathway/io/postgres/__init__.py +++ b/python/pathway/io/postgres/__init__.py @@ -31,8 +31,10 @@ def write( table: Table, postgres_settings: dict, table_name: str, + *, max_batch_size: int | None = None, init_mode: str = "default", + name: str | None = None, ) -> None: """Writes ``table``'s stream of updates to a postgres table. @@ -49,6 +51,8 @@ def write( "create_if_not_exists": initializes the SQL writer by creating the necessary table if they do not already exist; "replace": Initializes the SQL writer by replacing any existing table. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -124,7 +128,10 @@ def write( table.to( datasink.GenericDataSink( - data_storage, data_format, datasink_name="postgres.sink" + data_storage, + data_format, + datasink_name="postgres.sink", + unique_name=name, ) ) @@ -135,8 +142,10 @@ def write_snapshot( postgres_settings: dict, table_name: str, primary_key: list[str], + *, max_batch_size: int | None = None, init_mode: str = "default", + name: str | None = None, ) -> None: """Maintains a snapshot of a table within a Postgres table. @@ -153,7 +162,8 @@ def write_snapshot( "create_if_not_exists": initializes the SQL writer by creating the necessary table if they do not already exist; "replace": Initializes the SQL writer by replacing any existing table. - + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -215,6 +225,9 @@ def write_snapshot( table.to( datasink.GenericDataSink( - data_storage, data_format, datasink_name="postgres.snapshot" + data_storage, + data_format, + datasink_name="postgres.snapshot", + unique_name=name, ) ) diff --git a/python/pathway/io/pubsub/__init__.py b/python/pathway/io/pubsub/__init__.py index c45f73a7..9cfb0cd8 100644 --- a/python/pathway/io/pubsub/__init__.py +++ b/python/pathway/io/pubsub/__init__.py @@ -47,13 +47,18 @@ def _flush_publish_futures(self) -> None: def write( - table, publisher: pubsub_v1.PublisherClient, project_id: str, topic_id: str + table, + publisher: pubsub_v1.PublisherClient, + project_id: str, + topic_id: str, + *, + name: str | None = None, ) -> None: - """Publish the ``table``'s stream of changes into the specified PubSub topic. Please note \ -that ``table`` must consist of a single column of the binary type. In addition, the connector \ -adds two attributes: ``pathway_time`` containing the logical time of the change and \ -``pathway_diff`` corresponding to the change type: either addition (``pathway_diff = 1``) \ -or deletion (``pathway_diff = -1``). + """Publish the ``table``'s stream of changes into the specified PubSub topic. Please note + that ``table`` must consist of a single column of the binary type. In addition, the connector + adds two attributes: ``pathway_time`` containing the logical time of the change and + ``pathway_diff`` corresponding to the change type: either addition (``pathway_diff = 1``) + or deletion (``pathway_diff = -1``). Args: table: The table to publish. @@ -67,6 +72,8 @@ def write( `_. project_id: The ID of the project where the changes are published. topic_id: The topic ID where the changes are published. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -119,5 +126,8 @@ def write( output_buffer = _OutputBuffer(publisher, project_id, topic_id) subscribe( - table, on_change=output_buffer.on_change, on_time_end=output_buffer.on_time_end + table, + on_change=output_buffer.on_change, + on_time_end=output_buffer.on_time_end, + name=name, ) diff --git a/python/pathway/io/pyfilesystem/__init__.py b/python/pathway/io/pyfilesystem/__init__.py index c5ec1378..2a2dc26b 100644 --- a/python/pathway/io/pyfilesystem/__init__.py +++ b/python/pathway/io/pyfilesystem/__init__.py @@ -30,7 +30,7 @@ class _PyFilesystemSubject(ConnectorSubject): def __init__( self, source, *, path, mode, refresh_interval, with_metadata, **kwargs ): - super().__init__(**kwargs) + super().__init__(datasource_name="pyfilesystem", **kwargs) self.source = source self.path = path self.mode = mode @@ -146,37 +146,42 @@ def read( refresh_interval: float = 30, mode: str = "streaming", with_metadata: bool = False, + name: str | None = None, ) -> Table: - """Reads a table from \ -`PyFilesystem _` source. + """Reads a table from + `PyFilesystem _` source. - It returns a table with a single column `data` containing each file in a binary \ -format. If the `with_metadata` option is specified, it also attaches a column \ -`_metadata` containing the metadata of the objects read. + It returns a table with a single column ``data`` containing each file in a binary + format. If the ``with_metadata`` option is specified, it also attaches a column + ``_metadata`` containing the metadata of the objects read. Args: source: PyFilesystem source. - path: Path inside the PyFilesystem source to process. All files within this \ -path will be processed recursively. If unspecified, the root of the source is taken. - mode: denotes how the engine polls the new data from the source. Currently \ -"streaming" and "static" are supported. If set to "streaming", it will check for \ -updates, deletions, and new files every `refresh_interval` seconds. "static" mode will \ -only consider the available data and ingest all of it in one commit. \ -The default value is "streaming". - refresh_interval: time in seconds between scans. Applicable if the mode is \ -set to "streaming". - with_metadata: when set to True, the connector will add column \ -named `_metadata` to the table. This column will contain file metadata, such as: \ -`path`, `name`, `owner`, `created_at`, `modified_at`, `accessed_at`, `size`. + path: Path inside the PyFilesystem source to process. All files within this + path will be processed recursively. If unspecified, the root of the source is taken. + mode: denotes how the engine polls the new data from the source. Currently + "streaming" and "static" are supported. If set to "streaming", it will check for + updates, deletions, and new files every ``refresh_interval`` seconds. "static" mode will + only consider the available data and ingest all of it in one commit. + The default value is "streaming". + refresh_interval: time in seconds between scans. Applicable if the mode is + set to "streaming". + with_metadata: when set to True, the connector will add column + named ``_metadata`` to the table. This column will contain file metadata, such as: + ``path``, ``name``, ``owner``, ``created_at``, ``modified_at``, ``accessed_at``, + ``size``. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Returns: The table read. Example: - Suppose that you want to read a file from a ZIP archive `projects.zip` with the \ -usage of PyFilesystem. To do that, you first need to import the `fs` library or just \ -the `open_fs` method and to create the data source. It can be done as follows: + Suppose that you want to read a file from a ZIP archive ``projects.zip`` with the + usage of PyFilesystem. To do that, you first need to import the `fs` library or just + the ``open_fs`` method and to create the data source. It can be done as follows: >>> from fs import open_fs >>> source = open_fs("zip://projects.zip") # doctest: +SKIP @@ -187,19 +192,19 @@ def read( ... >>> table = pw.io.pyfilesystem.read(source) # doctest: +SKIP - This command reads all files in the archive in full. If the data is not supposed to \ -be changed, it makes sense to run this read in the static mode. It can be done by \ -specifying the `mode` parameter: + This command reads all files in the archive in full. If the data is not supposed to + be changed, it makes sense to run this read in the static mode. It can be done by + specifying the ``mode`` parameter: >>> table = pw.io.pyfilesystem.read(source, mode="static") # doctest: +SKIP - Please note that PyFilesystem offers a great variety of sources that can be \ -read. You can refer to the \ -`"Index of Filesystems" _` \ -web page for the list and the respective documentation. + Please note that PyFilesystem offers a great variety of sources that can be + read. You can refer to the + `"Index of Filesystems" _` + web page for the list and the respective documentation. - For instance, you can also read a dataset from the remote FTP source with this \ -connector. It can be done with the usage of `FTP` file source with the code as follows: + For instance, you can also read a dataset from the remote FTP source with this + connector. It can be done with the usage of ``FTP`` file source with the code as follows: >>> source = fs.open_fs('ftp://login:password@ftp.example.com/datasets') # doctest: +SKIP >>> table = pw.io.pyfilesystem.read(source) # doctest: +SKIP @@ -217,5 +222,6 @@ def read( subject, format="binary", autocommit_duration_ms=None, - name="pyfilesystem", + name=name, + _stacklevel=5, ) diff --git a/python/pathway/io/python/__init__.py b/python/pathway/io/python/__init__.py index 09f479e8..683f3ffc 100644 --- a/python/pathway/io/python/__init__.py +++ b/python/pathway/io/python/__init__.py @@ -27,6 +27,7 @@ MetadataSchema, PlaintextDataSchema, RawDataSchema, + _get_unique_name, assert_schema_or_value_columns_not_none, get_data_format_type, internal_read_method, @@ -96,13 +97,15 @@ class ConnectorSubject(ABC): _exception: BaseException | None _already_used: bool _pw_format: str + _datasource_name: str - def __init__(self) -> None: + def __init__(self, datasource_name: str = "python") -> None: self._buffer = Queue() self._thread = None self._exception = None self._already_used = False self._started = False + self._datasource_name = datasource_name @abstractmethod def run(self) -> None: ... @@ -329,7 +332,7 @@ def _is_internal(self) -> bool: We need this distinction, because internal usages don't read user data and aren't a part of the external perimeter, which is currently persisted. Therefore - we need to tell the engine not to require persistent_id from such connectors + we need to tell the engine not to require name from such connectors and not to store the snapshot of its inputs. """ return False @@ -360,8 +363,9 @@ def read( primary_key: list[str] | None = None, types: dict[str, PathwayType] | None = None, default_values: dict[str, Any] | None = None, - persistent_id: str | None = None, - name: str = "python", + name: str | None = None, + _stacklevel: int = 1, + **kwargs, ) -> Table: """Reads a table from a ConnectorSubject. @@ -385,11 +389,9 @@ def read( default_values: dictionary containing default values for columns replacing blank entries. The default value of the column must be specified explicitly, otherwise there will be no default value. [will be deprecated soon] - persistent_id: (unstable) An identifier, under which the state of the table \ -will be persisted or ``None``, if there is no need to persist the state of this table. \ -When a program restarts, it restores the state for all input tables according to what \ -was saved for their ``persistent_id``. This way it's possible to configure the start of \ -computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Returns: Table: The table read. @@ -433,7 +435,7 @@ def read( primary_key=primary_key, types=types, default_values=default_values, - _stacklevel=5, + _stacklevel=_stacklevel + 4, ) data_format = api.DataFormat( **api_schema, @@ -452,10 +454,10 @@ def read( deletions_enabled=subject._deletions_enabled, ), read_method=internal_read_method(format), - persistent_id=persistent_id, ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs, stacklevel=_stacklevel + 5), ) return table_from_datasource( datasource.GenericDataSource( @@ -463,7 +465,7 @@ def read( dataformat=data_format, data_source_options=data_source_options, schema=schema, - datasource_name=name, + datasource_name=subject._datasource_name, append_only=not subject._deletions_enabled, ), debug_datasource=datasource.debug_datasource(debug_data), diff --git a/python/pathway/io/redpanda/__init__.py b/python/pathway/io/redpanda/__init__.py index 74baf3df..de880d4b 100644 --- a/python/pathway/io/redpanda/__init__.py +++ b/python/pathway/io/redpanda/__init__.py @@ -25,7 +25,7 @@ def read( autocommit_duration_ms: int | None = 1500, json_field_paths: dict[str, str] | None = None, parallel_readers: int | None = None, - persistent_id: str | None = None, + name: str | None = None, value_columns: list[str] | None = None, primary_key: list[str] | None = None, types: dict[str, PathwayType] | None = None, @@ -55,11 +55,9 @@ def read( will be taken. This number also can't be greater than the number of Pathway engine threads, and will be reduced to the number of engine threads, if it exceeds. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. value_columns: Columns to extract for a table, required for format other than "raw". [will be deprecated soon] primary_key: In case the table should have a primary key generated according to @@ -92,8 +90,8 @@ def read( ... "session.timeout.ms": "60000" ... } - To connect to the topic "animals" and accept messages, the connector must be used \ - as follows, depending on the format: + To connect to the topic "animals" and accept messages, the connector must be used + as follows, depending on the format: Raw version: @@ -218,7 +216,7 @@ def read( autocommit_duration_ms=autocommit_duration_ms, json_field_paths=json_field_paths, parallel_readers=parallel_readers, - persistent_id=persistent_id, + name=name, _stacklevel=5, ) @@ -231,6 +229,7 @@ def write( topic_name: str, *, format: str = "json", + name: str | None = None, **kwargs, ) -> None: """Write a table to a given topic on a Redpanda instance. @@ -241,6 +240,8 @@ def write( `librdkafka `_. topic_name: name of topic in Redpanda to which the data should be sent. format: format of the input data, only "json" is currently supported. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Returns: None @@ -261,7 +262,7 @@ def write( ... "bootstrap.servers": "localhost:9092", ... "security.protocol": "sasl_ssl", ... "sasl.mechanism": "SCRAM-SHA-256", - ... "sasl.username": os.environ["KAFKA_USERNAME"], + ... "sasl.username": os.environ["KAFKA_USERNAME"], ... "sasl.password": os.environ["KAFKA_PASSWORD"] ... } @@ -270,8 +271,8 @@ def write( >>> import pathway as pw >>> t = pw.debug.table_from_markdown("age owner pet \\n 1 10 Alice dog \\n 2 9 Bob cat \\n 3 8 Alice cat") - To connect to the topic "animals" and send messages, the connector must be used \ - as follows, depending on the format: + To connect to the topic "animals" and send messages, the connector must be used + as follows, depending on the format: JSON version: @@ -290,5 +291,6 @@ def write( rdkafka_settings=rdkafka_settings, topic_name=topic_name, format=format, + name=name, **kwargs, ) diff --git a/python/pathway/io/s3/__init__.py b/python/pathway/io/s3/__init__.py index f4b94817..f2901b45 100644 --- a/python/pathway/io/s3/__init__.py +++ b/python/pathway/io/s3/__init__.py @@ -13,6 +13,7 @@ from pathway.internals.trace import trace_user_frame from pathway.io._utils import ( CsvParserSettings, + _get_unique_name, construct_s3_data_storage, construct_schema_and_data_format, internal_connector_mode, @@ -102,9 +103,11 @@ def read( csv_settings: CsvParserSettings | None = None, json_field_paths: dict[str, str] | None = None, downloader_threads_count: int | None = None, - persistent_id: str | None = None, + name: str | None = None, autocommit_duration_ms: int | None = 1500, debug_data: Any = None, + _stacklevel: int = 1, + **kwargs, ) -> Table: """Reads a table from one or several objects in Amazon S3 bucket in the given format. @@ -147,11 +150,9 @@ def read( of the bucket under the given path. It defaults to the number of cores available on the machine. It is recommended to increase the number of threads if your bucket contains many small files. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -272,7 +273,6 @@ def read( format=format, mode=internal_mode, csv_settings=csv_settings, - persistent_id=persistent_id, downloader_threads_count=downloader_threads_count, ) @@ -282,10 +282,11 @@ def read( csv_settings=csv_settings, json_field_paths=json_field_paths, with_metadata=with_metadata, - _stacklevel=5, + _stacklevel=_stacklevel + 4, ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs, stacklevel=_stacklevel + 5), ) return table_from_datasource( datasource.GenericDataSource( @@ -312,9 +313,10 @@ def read_from_digital_ocean( csv_settings: CsvParserSettings | None = None, json_field_paths: dict[str, str] | None = None, downloader_threads_count: int | None = None, - persistent_id: str | None = None, + name: str | None = None, autocommit_duration_ms: int | None = 1500, debug_data: Any = None, + **kwargs, ) -> Table: """Reads a table from one or several objects in Digital Ocean S3 bucket. @@ -356,11 +358,9 @@ def read_from_digital_ocean( of the bucket under the given path. It defaults to the number of cores available on the machine. It is recommended to increase the number of threads if your bucket contains many small files. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -403,7 +403,6 @@ def read_from_digital_ocean( format=format, mode=internal_mode, csv_settings=csv_settings, - persistent_id=persistent_id, downloader_threads_count=downloader_threads_count, ) @@ -416,7 +415,8 @@ def read_from_digital_ocean( _stacklevel=5, ) datasource_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs), ) return table_from_datasource( datasource.GenericDataSource( @@ -443,9 +443,10 @@ def read_from_wasabi( csv_settings: CsvParserSettings | None = None, json_field_paths: dict[str, str] | None = None, downloader_threads_count: int | None = None, - persistent_id: str | None = None, + name: str | None = None, autocommit_duration_ms: int | None = 1500, debug_data: Any = None, + **kwargs, ) -> Table: """Reads a table from one or several objects in Wasabi S3 bucket. @@ -486,11 +487,9 @@ def read_from_wasabi( of the bucket under the given path. It defaults to the number of cores available on the machine. It is recommended to increase the number of threads if your bucket contains many small files. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. Returns: @@ -534,7 +533,6 @@ def read_from_wasabi( mode=internal_mode, downloader_threads_count=downloader_threads_count, csv_settings=csv_settings, - persistent_id=persistent_id, ) schema, data_format = construct_schema_and_data_format( format, @@ -545,7 +543,8 @@ def read_from_wasabi( _stacklevel=5, ) datasource_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=_get_unique_name(name, kwargs), ) return table_from_datasource( datasource.GenericDataSource( diff --git a/python/pathway/io/s3_csv/__init__.py b/python/pathway/io/s3_csv/__init__.py index 212f2251..08acd4a7 100644 --- a/python/pathway/io/s3_csv/__init__.py +++ b/python/pathway/io/s3_csv/__init__.py @@ -23,7 +23,7 @@ def read( csv_settings: CsvParserSettings | None = None, mode: str = "streaming", autocommit_duration_ms: int | None = 1500, - persistent_id: str | None = None, + name: str | None = None, debug_data=None, value_columns: list[str] | None = None, id_columns: list[str] | None = None, @@ -50,11 +50,9 @@ def read( autocommit_duration_ms: the maximum time between two commits. Every autocommit_duration_ms milliseconds, the updates received by the connector are committed and pushed into Pathway's computation graph. - persistent_id: (unstable) An identifier, under which the state of the table - will be persisted or ``None``, if there is no need to persist the state of this table. - When a program restarts, it restores the state for all input tables according to what - was saved for their ``persistent_id``. This way it's possible to configure the start of - computations from the moment they were terminated last time. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. debug_data: Static data replacing original one when debug mode is active. value_columns: Names of the columns to be extracted from the files. [will be deprecated soon] id_columns: In case the table should have a primary key generated according to @@ -114,8 +112,8 @@ def read( ... schema=InputSchema, ... ) - In case you are dealing with custom S3 buckets, there are \ -`two ways `_ + In case you are dealing with custom S3 buckets, there are + `two ways `_ to work with paths in requests. The default and the one used by AWS S3 is a virtually hosted-style. However, some installations of S3 in, for example, min.io do require to use of path-style requests. If this is the case, you can use the @@ -159,8 +157,9 @@ def read( schema=schema, mode=mode, csv_settings=csv_settings, - persistent_id=persistent_id, + name=name, autocommit_duration_ms=autocommit_duration_ms, debug_data=debug_data, + _stacklevel=5, **kwargs, ) diff --git a/python/pathway/io/sqlite/__init__.py b/python/pathway/io/sqlite/__init__.py index 15fb30a1..7cf0e379 100644 --- a/python/pathway/io/sqlite/__init__.py +++ b/python/pathway/io/sqlite/__init__.py @@ -22,6 +22,7 @@ def read( schema: type[Schema], *, autocommit_duration_ms: int | None = 1500, + name: str | None = None, debug_data: Any = None, ) -> Table: """Reads a table from a rowid table in `SQLite `_ database. @@ -33,6 +34,9 @@ def read( autocommit_duration_ms: The maximum time between two commits. Every autocommit_duration_ms milliseconds, the updates received by the connector are committed and pushed into Pathway's computation graph. + name: A unique name for the connector. If provided, this name will be used in + logs and monitoring dashboards. Additionally, if persistence is enabled, it + will be used as the name for the snapshot that stores the connector's progress. Returns: Table: The table read. @@ -57,7 +61,8 @@ def read( ) data_source_options = datasource.DataSourceOptions( - commit_duration_ms=autocommit_duration_ms + commit_duration_ms=autocommit_duration_ms, + unique_name=name, ) return table_from_datasource( datasource.GenericDataSource( diff --git a/python/pathway/stdlib/utils/async_transformer.py b/python/pathway/stdlib/utils/async_transformer.py index 40b94362..11c0d030 100644 --- a/python/pathway/stdlib/utils/async_transformer.py +++ b/python/pathway/stdlib/utils/async_transformer.py @@ -70,7 +70,7 @@ class _AsyncConnector(io.python.ConnectorSubject): _logger: logging.Logger def __init__(self, transformer: AsyncTransformer) -> None: - super().__init__() + super().__init__(datasource_name="async-transformer") self._transformer = transformer self._event_loop = asyncio.new_event_loop() self._logger = logging.getLogger(__name__) @@ -491,7 +491,6 @@ def output_table(self) -> pw.Table: table: pw.Table = io.python.read( self._connector, schema=schema | _AsyncStatusSchema, - name="async-transformer", autocommit_duration_ms=self._autocommit_duration_ms, ) input_node = table._source.operator diff --git a/python/pathway/tests/cli/replay.py b/python/pathway/tests/cli/replay.py index 957b2bbc..1e36f38e 100644 --- a/python/pathway/tests/cli/replay.py +++ b/python/pathway/tests/cli/replay.py @@ -24,7 +24,7 @@ class InputSchema(pw.Schema): nb_rows=rows_to_generate, input_rate=15, autocommit_duration_ms=50, - persistent_id="1", + name="1", ) callback = CountDifferentTimestampsCallback(expected_count) diff --git a/python/pathway/tests/test_deduplicate.py b/python/pathway/tests/test_deduplicate.py index d863ec47..b8c9ceea 100644 --- a/python/pathway/tests/test_deduplicate.py +++ b/python/pathway/tests/test_deduplicate.py @@ -397,7 +397,7 @@ def on_change(key: pw.Pointer, row: dict, time: int, is_addition: bool): run_computation(5, 6, [6, 8, 10]) -def test_selective_persistence_persistent_id_set( +def test_selective_persistence_name_set( tmp_path: pathlib.Path, ): persistence_path = tmp_path / "persistence" @@ -430,9 +430,7 @@ def acceptor(new_value, old_value) -> bool: return new_value >= old_value + 2 table = pw.debug.table_from_markdown(data_1) - result = table.deduplicate( - value=pw.this.val, acceptor=acceptor, persistent_id="foo" - ) + result = table.deduplicate(value=pw.this.val, acceptor=acceptor, name="foo") expected_1 = pw.debug.table_from_markdown( """ @@ -450,9 +448,7 @@ def acceptor(new_value, old_value) -> bool: G.clear() table = pw.debug.table_from_markdown(data_2) - result = table.deduplicate( - value=pw.this.val, acceptor=acceptor, persistent_id="foo" - ) + result = table.deduplicate(value=pw.this.val, acceptor=acceptor, name="foo") expected_2 = pw.debug.table_from_markdown( """ @@ -472,7 +468,7 @@ def acceptor(new_value, old_value) -> bool: @pytest.mark.parametrize( "first_id,second_id", [(None, None), ("foo", "bar"), (None, "foo"), ("bar", None)] ) -def test_selective_persistence_no_persistent_id_set_or_different_ids_set( +def test_selective_persistence_no_name_set_or_different_names_set( tmp_path: pathlib.Path, first_id: str | None, second_id: str | None, @@ -507,9 +503,7 @@ def acceptor(new_value, old_value) -> bool: return new_value >= old_value + 2 table = pw.debug.table_from_markdown(data_1) - result = table.deduplicate( - value=pw.this.val, acceptor=acceptor, persistent_id=first_id - ) + result = table.deduplicate(value=pw.this.val, acceptor=acceptor, name=first_id) expected_1 = pw.debug.table_from_markdown( """ @@ -527,9 +521,7 @@ def acceptor(new_value, old_value) -> bool: G.clear() table = pw.debug.table_from_markdown(data_2) - result = table.deduplicate( - value=pw.this.val, acceptor=acceptor, persistent_id=second_id - ) + result = table.deduplicate(value=pw.this.val, acceptor=acceptor, name=second_id) expected_2 = pw.debug.table_from_markdown( """ diff --git a/python/pathway/tests/test_io.py b/python/pathway/tests/test_io.py index 2b8166b9..12ebc560 100644 --- a/python/pathway/tests/test_io.py +++ b/python/pathway/tests/test_io.py @@ -1019,9 +1019,9 @@ def run_computation(py_connector_input, fs_connector_input): G.clear() write_lines(input_path, "\n".join(fs_connector_input)) table_py = pw.io.python.read( - TestSubject(py_connector_input), format="raw", persistent_id="1" + TestSubject(py_connector_input), format="raw", name="1" ) - table_csv = pw.io.plaintext.read(input_path, persistent_id="2", mode="static") + table_csv = pw.io.plaintext.read(input_path, name="2", mode="static") table_joined = table_py.join(table_csv, table_py.data == table_csv.data).select( table_py.data ) @@ -1081,7 +1081,7 @@ def test_no_pstorage(tmp_path: pathlib.Path): ) -def test_persistent_id_not_assigned_autogenerate(tmp_path: pathlib.Path): +def test_name_not_assigned_autogenerate(tmp_path: pathlib.Path): input_path = tmp_path / "input.txt" write_lines(input_path, "test_data") pstorage_path = tmp_path / "PStrorage" @@ -1097,20 +1097,16 @@ def test_persistent_id_not_assigned_autogenerate(tmp_path: pathlib.Path): ) -def test_no_persistent_storage(tmp_path: pathlib.Path): +def test_no_persistent_storage_has_name(tmp_path: pathlib.Path): input_path = tmp_path / "input.txt" write_lines(input_path, "test_data") - table = pw.io.plaintext.read(input_path, persistent_id="1") + table = pw.io.plaintext.read(input_path, name="1", mode="static") pw.io.csv.write(table, tmp_path / "output.txt") - with pytest.raises( - ValueError, - match="persistent id 1 is assigned, but no persistent storage is configured", - ): - run() + run() -def test_duplicated_persistent_id(tmp_path: pathlib.Path): +def test_duplicated_name(tmp_path: pathlib.Path): pstorage_path = tmp_path / "PStorage" input_path = tmp_path / "input_first.txt" input_path_2 = tmp_path / "input_second.txt" @@ -1119,15 +1115,60 @@ def test_duplicated_persistent_id(tmp_path: pathlib.Path): write_lines(input_path, "hello") write_lines(input_path_2, "world") - table_1 = pw.io.plaintext.read(input_path, persistent_id="one") - table_2 = pw.io.plaintext.read(input_path_2, persistent_id="one") + table_1 = pw.io.plaintext.read(input_path, name="one") + table_2 = pw.io.plaintext.read(input_path_2, name="one") pw.universes.promise_are_pairwise_disjoint(table_1, table_2) table_concat = table_1.concat(table_2) pw.io.csv.write(table_concat, output_path) with pytest.raises( ValueError, - match="Persistent ID 'one' used more than once", + match="Unique name 'one' used more than once", + ): + run( + persistence_config=pw.persistence.Config( + pw.persistence.Backend.filesystem(pstorage_path) + ) + ) + + +def test_duplicated_name_between_input_and_output(tmp_path: pathlib.Path): + pstorage_path = tmp_path / "PStorage" + input_path = tmp_path / "input_first.txt" + input_path_2 = tmp_path / "input_second.txt" + output_path = tmp_path / "output.txt" + + write_lines(input_path, "hello") + write_lines(input_path_2, "world") + + table = pw.io.plaintext.read(input_path, name="one") + pw.io.jsonlines.write(table, output_path, name="one") + + with pytest.raises( + ValueError, + match="Unique name 'one' used more than once", + ): + run( + persistence_config=pw.persistence.Config( + pw.persistence.Backend.filesystem(pstorage_path) + ) + ) + + +def test_duplicated_name_between_input_and_subscribe(tmp_path: pathlib.Path): + pstorage_path = tmp_path / "PStorage" + input_path = tmp_path / "input_first.txt" + input_path_2 = tmp_path / "input_second.txt" + + write_lines(input_path, "hello") + write_lines(input_path_2, "world") + + table = pw.io.plaintext.read(input_path, name="one") + pw.io.subscribe(table, lambda **kwargs: print(kwargs), name="one") + + with pytest.raises( + ValueError, + match="Unique name 'one' used more than once", ): run( persistence_config=pw.persistence.Config( @@ -1585,7 +1626,7 @@ class TestSchema(pw.Schema): table = pw.io.csv.read( str(input_path), schema=TestSchema, - persistent_id="1", + name="1", mode="static", ) @@ -1624,7 +1665,7 @@ class TestSchema(pw.Schema): table = pw.io.csv.read( str(input_path), schema=TestSchema, - persistent_id="1", + name="1", mode="static", ) @@ -1726,7 +1767,7 @@ def run_graph( nb_rows=generate_rows, input_rate=15, autocommit_duration_ms=50, - persistent_id="1", + name="1", ) callback = CollectValuesCallback(expected, "number") @@ -1824,7 +1865,7 @@ def run_graph( nb_rows=generate_rows, input_rate=15, autocommit_duration_ms=50, - persistent_id="1", + name="1", ) callback = CountDifferentTimestampsCallback(expected_count) @@ -1951,7 +1992,7 @@ class InputSchema(pw.Schema): nb_rows=0, input_rate=15, autocommit_duration_ms=50, - persistent_id="1", + name="1", ) on_change = mock.Mock() diff --git a/python/pathway/tests/test_persistence.py b/python/pathway/tests/test_persistence.py index 47212e9c..906e9cba 100644 --- a/python/pathway/tests/test_persistence.py +++ b/python/pathway/tests/test_persistence.py @@ -620,8 +620,8 @@ def logic(t_1: pw.Table) -> pw.Table: "mode", [api.PersistenceMode.PERSISTING, api.PersistenceMode.OPERATOR_PERSISTING], ) -@pytest.mark.parametrize("persistent_id", [None, "ded"]) -def test_deduplicate(tmp_path, mode, persistent_id): +@pytest.mark.parametrize("name", [None, "ded"]) +def test_deduplicate(tmp_path, mode, name): class InputSchema(pw.Schema): a: int @@ -629,9 +629,7 @@ def acceptor(new_value, old_value) -> bool: return new_value > old_value + 2 def logic(t_1: pw.Table) -> pw.Table: - return t_1.deduplicate( - value=pw.this.a, acceptor=acceptor, persistent_id=persistent_id - ) + return t_1.deduplicate(value=pw.this.a, acceptor=acceptor, name=name) run, _ = get_one_table_runner(tmp_path, mode, logic, InputSchema) diff --git a/python/pathway/xpacks/connectors/sharepoint/__init__.py b/python/pathway/xpacks/connectors/sharepoint/__init__.py index 3ba3c390..3ebbb556 100644 --- a/python/pathway/xpacks/connectors/sharepoint/__init__.py +++ b/python/pathway/xpacks/connectors/sharepoint/__init__.py @@ -156,7 +156,7 @@ def __init__( max_failed_attempts_in_row, ): _check_entitlements("xpack-sharepoint") - super().__init__() + super().__init__(datasource_name="sharepoint") self._context_wrapper = context_wrapper self._root_path = root_path self._refresh_interval = refresh_interval @@ -373,4 +373,4 @@ def read( max_failed_attempts_in_row=max_failed_attempts_in_row, ) - return pw.io.python.read(subject, format="binary", name="sharepoint") + return pw.io.python.read(subject, format="binary") diff --git a/src/connectors/data_lake/delta.rs b/src/connectors/data_lake/delta.rs index 61f8f270..b045602e 100644 --- a/src/connectors/data_lake/delta.rs +++ b/src/connectors/data_lake/delta.rs @@ -1,4 +1,5 @@ use log::{info, warn}; +use std::borrow::Cow; use std::collections::{HashMap, VecDeque}; use std::fs::File; use std::io::{Seek, SeekFrom, Write}; @@ -37,7 +38,6 @@ use crate::connectors::{ }; use crate::engine::Type; use crate::persistence::frontier::OffsetAntichain; -use crate::persistence::PersistentId; use crate::python_api::ValueField; #[allow(clippy::module_name_repetitions)] @@ -187,6 +187,10 @@ impl LakeBatchWriter for DeltaBatchWriter { utc_timezone_name: "UTC".into(), } } + + fn name(&self) -> String { + format!("DeltaTable({})", self.table.table_uri()) + } } pub enum ObjectDownloader { @@ -229,7 +233,6 @@ pub struct DeltaTableReader { table: DeltaTable, streaming_mode: ConnectorMode, column_types: HashMap, - persistent_id: Option, base_path: String, object_downloader: ObjectDownloader, @@ -252,7 +255,6 @@ impl DeltaTableReader { storage_options: HashMap, column_types: HashMap, streaming_mode: ConnectorMode, - persistent_id: Option, ) -> Result { let runtime = create_async_tokio_runtime()?; let table = runtime.block_on(async { open_delta_table(path, storage_options).await })?; @@ -263,7 +265,6 @@ impl DeltaTableReader { table, column_types, streaming_mode, - persistent_id, base_path: path.to_string(), current_version, @@ -480,12 +481,8 @@ impl Reader for DeltaTableReader { Ok(()) } - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; - } - - fn persistent_id(&self) -> Option { - self.persistent_id + fn short_description(&self) -> Cow<'static, str> { + format!("DeltaTable({})", self.base_path).into() } fn storage_type(&self) -> StorageType { diff --git a/src/connectors/data_lake/iceberg.rs b/src/connectors/data_lake/iceberg.rs index 1af7acd9..32f0b74f 100644 --- a/src/connectors/data_lake/iceberg.rs +++ b/src/connectors/data_lake/iceberg.rs @@ -1,4 +1,5 @@ use log::warn; +use std::borrow::Cow; use std::collections::{HashMap, HashSet, VecDeque}; use std::sync::Arc; use std::thread::sleep; @@ -37,7 +38,6 @@ use crate::connectors::{ }; use crate::engine::Type; use crate::persistence::frontier::OffsetAntichain; -use crate::persistence::PersistentId; use crate::python_api::ValueField; use crate::timestamp::current_unix_timestamp_ms; @@ -276,6 +276,14 @@ impl LakeBatchWriter for IcebergBatchWriter { utc_timezone_name: "+00:00".into(), } } + + fn name(&self) -> String { + format!( + "Iceberg({}, {})", + self.table_ident.namespace.to_url_string(), + self.table_ident.name + ) + } } /// Wrapper for `FileScanTask` that allows to compare them. @@ -303,7 +311,6 @@ pub type IcebergSnapshotId = i64; pub struct IcebergReader { catalog: RestCatalog, table_ident: TableIdent, - persistent_id: Option, column_types: HashMap, streaming_mode: ConnectorMode, @@ -322,7 +329,6 @@ impl IcebergReader { table_params: &IcebergTableParams, column_types: HashMap, streaming_mode: ConnectorMode, - persistent_id: Option, ) -> Result { let runtime = create_async_tokio_runtime()?; let catalog = db_params.create_catalog(); @@ -335,7 +341,6 @@ impl IcebergReader { Ok(Self { catalog, table_ident, - persistent_id, column_types, streaming_mode, @@ -521,12 +526,13 @@ impl Reader for IcebergReader { Ok(()) } - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; - } - - fn persistent_id(&self) -> Option { - self.persistent_id + fn short_description(&self) -> Cow<'static, str> { + format!( + "Iceberg({}, {})", + self.table_ident.namespace.to_url_string(), + self.table_ident.name + ) + .into() } fn storage_type(&self) -> StorageType { diff --git a/src/connectors/data_lake/mod.rs b/src/connectors/data_lake/mod.rs index 871ba6d6..858781c3 100644 --- a/src/connectors/data_lake/mod.rs +++ b/src/connectors/data_lake/mod.rs @@ -46,6 +46,8 @@ pub trait LakeBatchWriter: Send { fn write_batch(&mut self, batch: ArrowRecordBatch) -> Result<(), WriteError>; fn settings(&self) -> LakeWriterSettings; + + fn name(&self) -> String; } type ParsedValue = Result>; diff --git a/src/connectors/data_lake/writer.rs b/src/connectors/data_lake/writer.rs index 3d5f3bc9..75d836cd 100644 --- a/src/connectors/data_lake/writer.rs +++ b/src/connectors/data_lake/writer.rs @@ -413,4 +413,8 @@ impl Writer for LakeWriter { } Ok(()) } + + fn name(&self) -> String { + self.batch_writer.name() + } } diff --git a/src/connectors/data_storage.rs b/src/connectors/data_storage.rs index a8b4727c..9ea0a849 100644 --- a/src/connectors/data_storage.rs +++ b/src/connectors/data_storage.rs @@ -52,7 +52,7 @@ use crate::engine::Value; use crate::persistence::backends::Error as PersistenceBackendError; use crate::persistence::frontier::OffsetAntichain; use crate::persistence::tracker::WorkerPersistentStorage; -use crate::persistence::{ExternalPersistentId, PersistentId}; +use crate::persistence::{PersistentId, UniqueName}; use crate::python_api::extract_value; use crate::python_api::threads::PythonThreadState; use crate::python_api::PythonSubject; @@ -303,9 +303,9 @@ impl StorageType { pub fn new_filesystem_reader( path: &str, streaming_mode: ConnectorMode, - persistent_id: Option, read_method: ReadMethod, object_pattern: &str, + is_persisted: bool, ) -> Result { let scanner = FilesystemScanner::new(path, object_pattern)?; let tokenizer = BufReaderTokenizer::new(read_method); @@ -313,7 +313,7 @@ pub fn new_filesystem_reader( Box::new(scanner), Box::new(tokenizer), streaming_mode, - persistent_id, + is_persisted, ) } @@ -321,8 +321,8 @@ pub fn new_csv_filesystem_reader( path: &str, parser_builder: csv::ReaderBuilder, streaming_mode: ConnectorMode, - persistent_id: Option, object_pattern: &str, + is_persisted: bool, ) -> Result { let scanner = FilesystemScanner::new(path, object_pattern)?; let tokenizer = CsvTokenizer::new(parser_builder); @@ -330,7 +330,7 @@ pub fn new_csv_filesystem_reader( Box::new(scanner), Box::new(tokenizer), streaming_mode, - persistent_id, + is_persisted, ) } @@ -338,9 +338,9 @@ pub fn new_s3_generic_reader( bucket: S3Bucket, objects_prefix: impl Into, streaming_mode: ConnectorMode, - persistent_id: Option, read_method: ReadMethod, downloader_threads_count: usize, + is_persisted: bool, ) -> Result { let scanner = S3Scanner::new(bucket, objects_prefix, downloader_threads_count)?; let tokenizer = BufReaderTokenizer::new(read_method); @@ -348,7 +348,7 @@ pub fn new_s3_generic_reader( Box::new(scanner), Box::new(tokenizer), streaming_mode, - persistent_id, + is_persisted, ) } @@ -357,8 +357,8 @@ pub fn new_s3_csv_reader( objects_prefix: impl Into, parser_builder: csv::ReaderBuilder, streaming_mode: ConnectorMode, - persistent_id: Option, downloader_threads_count: usize, + is_persisted: bool, ) -> Result { let scanner = S3Scanner::new(bucket, objects_prefix, downloader_threads_count)?; let tokenizer = CsvTokenizer::new(parser_builder); @@ -366,7 +366,7 @@ pub fn new_s3_csv_reader( Box::new(scanner), Box::new(tokenizer), streaming_mode, - persistent_id, + is_persisted, ) } @@ -376,8 +376,10 @@ pub trait Reader { #[allow(clippy::missing_errors_doc)] fn seek(&mut self, frontier: &OffsetAntichain) -> Result<(), ReadError>; - fn update_persistent_id(&mut self, persistent_id: Option); - fn persistent_id(&self) -> Option; + fn short_description(&self) -> Cow<'static, str> { + type_name::().into() + } + fn initialize_cached_objects_storage( &mut self, _: &WorkerPersistentStorage, @@ -486,28 +488,13 @@ pub trait Reader { pub trait ReaderBuilder: Send + 'static { fn build(self: Box) -> Result, ReadError>; - fn short_description(&self) -> Cow<'static, str> { - type_name::().into() - } - - fn name(&self, persistent_id: Option<&ExternalPersistentId>, id: usize) -> String { - let desc = self.short_description(); - let name = desc.split("::").last().unwrap().replace("Builder", ""); - if let Some(id) = persistent_id { - format!("{name}-{id}") - } else { - format!("{name}-{id}") - } - } - fn is_internal(&self) -> bool { false } - fn persistent_id(&self) -> Option; - fn update_persistent_id(&mut self, persistent_id: Option); - fn storage_type(&self) -> StorageType; + fn short_description(&self) -> Cow<'static, str>; + fn name(&self, unique_name: Option<&UniqueName>) -> String; } impl ReaderBuilder for T @@ -518,16 +505,21 @@ where Ok(self) } - fn persistent_id(&self) -> Option { - Reader::persistent_id(self) + fn storage_type(&self) -> StorageType { + Reader::storage_type(self) } - fn update_persistent_id(&mut self, persistent_id: Option) { - Reader::update_persistent_id(self, persistent_id); + fn short_description(&self) -> Cow<'static, str> { + Reader::short_description(self) } - fn storage_type(&self) -> StorageType { - Reader::storage_type(self) + fn name(&self, unique_name: Option<&UniqueName>) -> String { + if let Some(unique_name) = unique_name { + unique_name.to_string() + } else { + let desc = self.short_description(); + desc.split("::").last().unwrap().replace("Builder", "") + } } } @@ -613,28 +605,23 @@ pub trait Writer: Send { true } - fn short_description(&self) -> Cow<'static, str> { - type_name::().into() - } - - fn name(&self, id: usize) -> String { - let name = self - .short_description() - .split("::") - .last() - .unwrap() - .to_string(); - format!("{name}-{id}") + fn name(&self) -> String { + let short_description: Cow<'static, str> = type_name::().into(); + short_description.split("::").last().unwrap().to_string() } } pub struct FileWriter { writer: BufWriter, + output_path: String, } impl FileWriter { - pub fn new(writer: BufWriter) -> FileWriter { - FileWriter { writer } + pub fn new(writer: BufWriter, output_path: String) -> FileWriter { + FileWriter { + writer, + output_path, + } } } @@ -669,11 +656,14 @@ impl Writer for FileWriter { self.writer.flush()?; Ok(()) } + + fn name(&self) -> String { + format!("FileSystem({})", self.output_path) + } } pub struct KafkaReader { consumer: BaseConsumer, - persistent_id: Option, topic: ArcStr, positions_for_seek: HashMap, deferred_read_result: Option, @@ -767,12 +757,8 @@ impl Reader for KafkaReader { Ok(()) } - fn persistent_id(&self) -> Option { - self.persistent_id - } - - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; + fn short_description(&self) -> Cow<'static, str> { + format!("Kafka({})", self.topic).into() } fn storage_type(&self) -> StorageType { @@ -788,12 +774,10 @@ impl KafkaReader { pub fn new( consumer: BaseConsumer, topic: String, - persistent_id: Option, positions_for_seek: HashMap, ) -> KafkaReader { KafkaReader { consumer, - persistent_id, topic: topic.into(), positions_for_seek, deferred_read_result: None, @@ -818,13 +802,11 @@ impl ConnectorMode { pub struct PythonReaderBuilder { subject: Py, - persistent_id: Option, schema: HashMap, } pub struct PythonReader { subject: Py, - persistent_id: Option, schema: HashMap, total_entries_read: u64, current_external_offset: Arc<[u8]>, @@ -836,31 +818,18 @@ pub struct PythonReader { } impl PythonReaderBuilder { - pub fn new( - subject: Py, - persistent_id: Option, - schema: HashMap, - ) -> Self { - Self { - subject, - persistent_id, - schema, - } + pub fn new(subject: Py, schema: HashMap) -> Self { + Self { subject, schema } } } impl ReaderBuilder for PythonReaderBuilder { fn build(self: Box) -> Result, ReadError> { let python_thread_state = PythonThreadState::new(); - let Self { - subject, - persistent_id, - schema, - } = *self; + let Self { subject, schema } = *self; Ok(Box::new(PythonReader { subject, - persistent_id, schema, python_thread_state, total_entries_read: 0, @@ -870,16 +839,21 @@ impl ReaderBuilder for PythonReaderBuilder { })) } - fn is_internal(&self) -> bool { - self.subject.get().is_internal + fn short_description(&self) -> Cow<'static, str> { + type_name::().into() } - fn persistent_id(&self) -> Option { - self.persistent_id + fn name(&self, unique_name: Option<&UniqueName>) -> String { + if let Some(unique_name) = unique_name { + unique_name.to_string() + } else { + let desc = self.short_description(); + desc.split("::").last().unwrap().replace("Builder", "") + } } - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; + fn is_internal(&self) -> bool { + self.subject.get().is_internal } fn storage_type(&self) -> StorageType { @@ -1046,14 +1020,6 @@ impl Reader for PythonReader { }) } - fn persistent_id(&self) -> Option { - self.persistent_id - } - - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; - } - fn storage_type(&self) -> StorageType { StorageType::Python } @@ -1064,6 +1030,7 @@ pub struct PsqlWriter { max_batch_size: Option, buffer: Vec, snapshot_mode: bool, + table_name: String, } impl PsqlWriter { @@ -1081,6 +1048,7 @@ impl PsqlWriter { max_batch_size, buffer: Vec::new(), snapshot_mode, + table_name: table_name.to_string(), }; writer.initialize(mode, table_name, schema, key_field_names)?; Ok(writer) @@ -1351,6 +1319,10 @@ impl Writer for PsqlWriter { Ok(()) } + fn name(&self) -> String { + format!("Postgres({})", self.table_name) + } + fn single_threaded(&self) -> bool { self.snapshot_mode } @@ -1425,6 +1397,10 @@ impl Writer for KafkaWriter { Ok(()) } + fn name(&self) -> String { + format!("Kafka({})", self.topic) + } + fn retriable(&self) -> bool { true } @@ -1487,6 +1463,10 @@ impl Writer for ElasticSearchWriter { }) } + fn name(&self) -> String { + format!("ElasticSearch({})", self.index_name) + } + fn single_threaded(&self) -> bool { false } @@ -1702,18 +1682,12 @@ impl Reader for SqliteReader { } } - fn storage_type(&self) -> StorageType { - StorageType::Sqlite - } - - fn persistent_id(&self) -> Option { - None + fn short_description(&self) -> Cow<'static, str> { + format!("SQLite({})", self.table_name).into() } - fn update_persistent_id(&mut self, persistent_id: Option) { - if persistent_id.is_some() { - unimplemented!("persistence is not supported for Sqlite data source") - } + fn storage_type(&self) -> StorageType { + StorageType::Sqlite } } @@ -1754,14 +1728,18 @@ impl Writer for MongoWriter { let _ = command.run()?; Ok(()) } + + fn name(&self) -> String { + format!("MongoDB({})", self.collection.name()) + } } pub struct NatsReader { runtime: TokioRuntime, subscriber: NatsSubscriber, worker_index: usize, - persistent_id: Option, total_entries_read: usize, + stream_name: String, } impl Reader for NatsReader { @@ -1796,12 +1774,8 @@ impl Reader for NatsReader { Ok(()) } - fn persistent_id(&self) -> Option { - self.persistent_id - } - - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; + fn short_description(&self) -> Cow<'static, str> { + format!("NATS({})", self.stream_name).into() } fn storage_type(&self) -> StorageType { @@ -1818,13 +1792,13 @@ impl NatsReader { runtime: TokioRuntime, subscriber: NatsSubscriber, worker_index: usize, - persistent_id: Option, + stream_name: String, ) -> NatsReader { NatsReader { runtime, subscriber, worker_index, - persistent_id, + stream_name, total_entries_read: 0, } } @@ -1867,6 +1841,10 @@ impl Writer for NatsWriter { .map_err(WriteError::NatsFlush) } + fn name(&self) -> String { + format!("NATS({})", self.topic) + } + fn retriable(&self) -> bool { true } diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 218ef492..40186634 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -41,7 +41,7 @@ use crate::persistence::config::ReadersQueryPurpose; use crate::persistence::frontier::OffsetAntichain; use crate::persistence::input_snapshot::{Event as SnapshotEvent, SnapshotMode}; use crate::persistence::tracker::{RequiredPersistenceMode, WorkerPersistentStorage}; -use crate::persistence::{ExternalPersistentId, PersistentId, SharedSnapshotWriter}; +use crate::persistence::{PersistentId, SharedSnapshotWriter, UniqueName}; use data_format::{ParseError, ParseResult, ParsedEvent, ParsedEventWithErrors, Parser}; use data_storage::{ @@ -342,6 +342,7 @@ impl Connector { pub fn read_snapshot( reader: &mut dyn Reader, persistent_storage: Option<&Arc>>, + persistent_id: Option, sender: &Sender, persistence_mode: PersistenceMode, snapshot_access: SnapshotAccess, @@ -356,7 +357,7 @@ impl Connector { persistence_mode.on_before_reading_snapshot(sender); } if let Some(persistent_storage) = persistent_storage { - if let Some(persistent_id) = reader.persistent_id() { + if let Some(persistent_id) = persistent_id { reader.initialize_cached_objects_storage( &persistent_storage.lock().unwrap(), persistent_id, @@ -394,13 +395,14 @@ impl Connector { pub fn snapshot_writer( reader: &dyn ReaderBuilder, + persistent_id: Option, persistent_storage: Option<&Arc>>, snapshot_access: SnapshotAccess, ) -> Result, WriteError> { if !snapshot_access.is_snapshot_writing_allowed() { Ok(None) } else if let Some(persistent_storage) = &persistent_storage { - if let Some(persistent_id) = reader.persistent_id() { + if let Some(persistent_id) = persistent_id { Ok(Some( persistent_storage .lock() @@ -418,7 +420,6 @@ impl Connector { Ok(None) } } else { - assert!(reader.persistent_id().is_none()); Ok(None) } } @@ -433,9 +434,9 @@ impl Connector { mut values_to_key: impl FnMut(Option<&Vec>, Option<&Offset>) -> Key + 'static, probe: Handle, persistent_storage: Option>>, - connector_id: usize, + persistent_id: Option, realtime_reader_needed: bool, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, persistence_mode: PersistenceMode, snapshot_access: SnapshotAccess, error_reporter: impl ReportError + 'static, @@ -450,10 +451,11 @@ impl Connector { reader.short_description(), parser.short_description() ); - let reader_name = reader.name(external_persistent_id, connector_id); + let reader_name = reader.name(unique_name); let mut snapshot_writer = Self::snapshot_writer( reader.as_ref(), + persistent_id, persistent_storage.as_ref(), snapshot_access, ) @@ -473,6 +475,7 @@ impl Connector { Self::read_snapshot( &mut *reader, persistent_storage.as_ref(), + persistent_id, &sender, persistence_mode, snapshot_access, @@ -864,11 +867,11 @@ pub struct SnapshotReaderState { pub fn read_persisted_state( mut input_session: InputSession), isize>, persistent_storage: Arc>, - external_persistent_id: &ExternalPersistentId, + unique_name: &UniqueName, persistent_id: PersistentId, ) -> SnapshotReaderState { let (sender, receiver) = mpsc::channel(); - let thread_name = format!("pathway:{external_persistent_id}"); + let thread_name = format!("pathway:{unique_name}"); let input_thread_handle = thread::Builder::new() .name(thread_name) diff --git a/src/connectors/monitoring.rs b/src/connectors/monitoring.rs index 85aa4164..f10ae836 100644 --- a/src/connectors/monitoring.rs +++ b/src/connectors/monitoring.rs @@ -200,9 +200,9 @@ impl OutputConnectorStats { self.batch_start_time = Instant::now(); } - pub fn on_batch_entry_written(&mut self) { - self.messages_written_in_batch += 1; - self.messages_written_in_total += 1; + pub fn on_batch_entries_written(&mut self, n_entries: usize) { + self.messages_written_in_batch += n_entries; + self.messages_written_in_total += n_entries; } pub fn on_batch_finished(&mut self) { diff --git a/src/connectors/posix_like.rs b/src/connectors/posix_like.rs index 8cdfb15d..890e092f 100644 --- a/src/connectors/posix_like.rs +++ b/src/connectors/posix_like.rs @@ -1,6 +1,7 @@ // Copyright © 2024 Pathway use log::{error, info, warn}; +use std::borrow::Cow; use std::collections::VecDeque; use std::io::Cursor; use std::mem::take; @@ -38,8 +39,8 @@ impl From for CurrentAction { pub struct PosixLikeReader { scanner: Box, tokenizer: Box, - persistent_id: Option, streaming_mode: ConnectorMode, + is_persisted: bool, total_entries_read: u64, had_queue_refresh: bool, @@ -53,13 +54,13 @@ impl PosixLikeReader { scanner: Box, tokenizer: Box, streaming_mode: ConnectorMode, - persistent_id: Option, + is_persisted: bool, ) -> Result { Ok(Self { scanner, tokenizer, streaming_mode, - persistent_id, + is_persisted, total_entries_read: 0, had_queue_refresh: false, @@ -159,12 +160,8 @@ impl Reader for PosixLikeReader { Ok(ReadResult::Finished) } - fn persistent_id(&self) -> Option { - self.persistent_id - } - - fn update_persistent_id(&mut self, persistent_id: Option) { - self.persistent_id = persistent_id; + fn short_description(&self) -> Cow<'static, str> { + self.scanner.short_description().into() } fn storage_type(&self) -> StorageType { @@ -186,8 +183,7 @@ impl PosixLikeReader { } QueuedAction::Read(path, _) => { let are_deletions_enabled = self.are_deletions_enabled(); - let is_persisted = self.persistent_id.is_some(); - if !is_persisted && !are_deletions_enabled { + if !self.is_persisted && !are_deletions_enabled { // Don't store a copy in memory if it won't be // needed for undoing an object. self.cached_object_storage @@ -270,7 +266,7 @@ impl PosixLikeReader { } fn are_deletions_enabled(&self) -> bool { - self.persistent_id.is_some() || self.streaming_mode.is_polling_enabled() + self.is_persisted || self.streaming_mode.is_polling_enabled() } fn sleep_duration() -> Duration { diff --git a/src/connectors/scanner/filesystem.rs b/src/connectors/scanner/filesystem.rs index 5dbe7a84..c9a934e1 100644 --- a/src/connectors/scanner/filesystem.rs +++ b/src/connectors/scanner/filesystem.rs @@ -56,6 +56,10 @@ impl PosixLikeScanner for FilesystemScanner { result.append(&mut self.new_insertion_actions(cached_object_storage)?); Ok(result) } + + fn short_description(&self) -> String { + format!("FileSystem({})", self.path) + } } impl FilesystemScanner { diff --git a/src/connectors/scanner/mod.rs b/src/connectors/scanner/mod.rs index 5639e8a7..6913520a 100644 --- a/src/connectors/scanner/mod.rs +++ b/src/connectors/scanner/mod.rs @@ -38,4 +38,5 @@ pub trait PosixLikeScanner: Send { are_deletions_enabled: bool, cached_object_storage: &CachedObjectStorage, ) -> Result, ReadError>; + fn short_description(&self) -> String; } diff --git a/src/connectors/scanner/s3.rs b/src/connectors/scanner/s3.rs index 0d75e739..1d0b62b8 100644 --- a/src/connectors/scanner/s3.rs +++ b/src/connectors/scanner/s3.rs @@ -169,6 +169,10 @@ impl PosixLikeScanner for S3Scanner { } Ok(result) } + + fn short_description(&self) -> String { + format!("S3({})", self.objects_prefix) + } } #[allow(clippy::module_name_repetitions)] diff --git a/src/engine/dataflow.rs b/src/engine/dataflow.rs index 5730e5f8..41ddc8ea 100644 --- a/src/engine/dataflow.rs +++ b/src/engine/dataflow.rs @@ -26,7 +26,7 @@ use crate::engine::telemetry::Config as TelemetryConfig; use crate::engine::value::HashInto; use crate::persistence::config::PersistenceManagerOuterConfig; use crate::persistence::tracker::{RequiredPersistenceMode, SharedWorkerPersistentStorage}; -use crate::persistence::{ExternalPersistentId, IntoPersistentId, PersistenceTime}; +use crate::persistence::{IntoPersistentId, PersistenceTime, UniqueName}; use crate::retry::{execute_with_retries, RetryConfig}; use std::borrow::{Borrow, Cow}; @@ -1057,7 +1057,7 @@ where connector_threads: &mut Vec>, name: &str, ) -> Result { - // TODO: generate better persistent ids that can be used even if graph changes + // TODO: generate better unique names that can be used even if graph changes let effective_persistent_id = effective_persistent_id( persistence_wrapper, false, @@ -1065,10 +1065,10 @@ where RequiredPersistenceMode::OperatorPersistence, |next_state_id| { let generated_external_id = format!("{name}-{next_state_id}"); - info!("Persistent ID autogenerated for {name}: {generated_external_id}"); + info!("Unique name autogenerated for {name} because persistence is enabled: {generated_external_id}"); generated_external_id }, - )?; + ); let persistent_id = effective_persistent_id .clone() .map(IntoPersistentId::into_persistent_id); @@ -1101,7 +1101,7 @@ where None, RequiredPersistenceMode::OperatorPersistence, |_| String::new(), - )? + ) .is_some(); if with_persistent_id { Ok(persistence_wrapper @@ -1124,7 +1124,7 @@ where &self, graph: &mut DataflowGraphInner, name: &str, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, required_persistence_mode: RequiredPersistenceMode, logic: impl FnMut(Option<&V2>, Vec<(V, R)>) -> Option + 'static, ) -> Result> @@ -1145,7 +1145,7 @@ where &self, graph: &mut DataflowGraphInner, name: &str, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, required_persistence_mode: RequiredPersistenceMode, logic: impl FnMut(Option<&V2>, Vec<(V, R)>) -> Option + 'static, ) -> Result> @@ -1157,14 +1157,14 @@ where let effective_persistent_id = effective_persistent_id( &mut graph.persistence_wrapper, false, - external_persistent_id, + unique_name, required_persistence_mode, |next_state_id| { let generated_external_id = format!("{name}-{next_state_id}"); - info!("Persistent ID autogenerated for {name}: {generated_external_id}"); + info!("Unique name autogenerated for {name}: {generated_external_id}"); generated_external_id }, - )?; + ); let persistent_id = effective_persistent_id .clone() .map(IntoPersistentId::into_persistent_id); @@ -1222,10 +1222,10 @@ impl MaybePersistedMapWithDeletionsFirst RequiredPersistenceMode::OperatorPersistence, |next_state_id| { let generated_external_id = format!("{name}-{next_state_id}"); - info!("Persistent ID autogenerated for {name}: {generated_external_id}"); + info!("Unique name autogenerated for {name} because persistence is enabled: {generated_external_id}"); generated_external_id }, - )?; + ); let persistent_id = effective_persistent_id .clone() .map(IntoPersistentId::into_persistent_id); @@ -3514,7 +3514,7 @@ impl> DataflowGraphInner grouping_columns_paths: Vec, reduced_column_paths: Vec, combine_fn: StatefulCombineFn, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, table_properties: Arc, ) -> Result { let table = self @@ -3554,7 +3554,7 @@ impl> DataflowGraphInner .maybe_persisted_stateful_reduce( self, "deduplicate::reduce", - external_persistent_id, + unique_name, RequiredPersistenceMode::InputOrOperatorPersistence, move |state, values| match (combine_fn)(state, values) { Ok(new_state) => new_state, @@ -3615,29 +3615,30 @@ impl> DataflowGraphInner fn connector_table( &mut self, - mut reader: Box, + reader: Box, parser: Box, commit_duration: Option, parallel_readers: usize, table_properties: Arc, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, ) -> Result { let effective_persistent_id = effective_persistent_id( &mut self.persistence_wrapper, reader.is_internal(), - external_persistent_id, + unique_name, RequiredPersistenceMode::InputOrOperatorPersistence, |_| { - let generated_external_id = reader.name(None, self.connector_monitors.len()); - reader - .update_persistent_id(Some(generated_external_id.clone().into_persistent_id())); + let generated_external_id = reader.name(None); info!( - "Persistent ID autogenerated for a {:?} reader: {generated_external_id}", + "Unique name autogenerated for a {:?} reader because persistence is enabled: {generated_external_id}", reader.storage_type() ); generated_external_id }, - )?; + ); + let internal_persistent_id = effective_persistent_id + .clone() + .map(IntoPersistentId::into_persistent_id); let (input_session, table_values): (ValuesSessionAdaptor, GenericValues) = parser.session_type().new_collection(&mut self.scope); @@ -3650,14 +3651,14 @@ impl> DataflowGraphInner .persistence_wrapper .get_persistence_config() .map_or(true, |config| config.continue_after_replay); - let persisted_table = reader.persistent_id().is_some() + let persisted_table = internal_persistent_id.is_some() && self .persistence_wrapper .get_worker_persistent_storage() .is_some(); if realtime_reader_needed || persisted_table { - let persistent_id = reader.persistent_id(); + let persistent_id = internal_persistent_id; let persistence_mode = self .persistence_wrapper .get_persistence_config() @@ -3700,9 +3701,9 @@ impl> DataflowGraphInner self.persistence_wrapper .get_worker_persistent_storage() .cloned(), - self.connector_monitors.len(), + internal_persistent_id, realtime_reader_needed, - effective_persistent_id.as_ref(), + unique_name, persistence_mode, snapshot_access, self.error_reporter.clone(), @@ -3868,6 +3869,7 @@ impl> DataflowGraphInner ) -> Result<(), DynError> { stats.on_batch_started(); let time = batch.time; + let batch_size = batch.data.len(); for ((key, values), diff) in batch.data { if time.is_from_persistence() && worker_persistent_storage.is_some() { // Ignore entries, which had been written before @@ -3891,8 +3893,8 @@ impl> DataflowGraphInner RetryConfig::default(), retries, )?; - stats.on_batch_entry_written(); } + stats.on_batch_entries_written(batch_size); stats.on_batch_finished(); // This line can be removed. In this case, flush will happen on the next time advancement. @@ -3925,6 +3927,7 @@ impl> DataflowGraphInner mut data_formatter: Box, table_handle: TableHandle, column_paths: Vec, + unique_name: Option, ) -> Result<()> { let worker_index = self.scope.index(); let error_logger = self.create_error_logger()?; @@ -3947,7 +3950,7 @@ impl> DataflowGraphInner let thread_name = format!( "pathway:output_table-{}-{}", - data_sink.short_description(), + data_sink.name(), data_formatter.short_description() ); @@ -3956,11 +3959,8 @@ impl> DataflowGraphInner .get_worker_persistent_storage() .cloned(); - // connector_threads vector contains both, input and output connector threads - // connector_monitors vector contains monitors only for input connectors - let output_connector_id = self.connector_threads.len() - self.connector_monitors.len(); - let mut stats = OutputConnectorStats::new(data_sink.name(output_connector_id)); - + let stats_name = unique_name.unwrap_or(data_sink.name()); + let mut stats = OutputConnectorStats::new(stats_name); let output_joiner_handle = Builder::new() .name(thread_name) .spawn_with_reporter( @@ -4029,6 +4029,7 @@ impl> DataflowGraphInner callbacks: SubscribeCallbacks, skip_persisted_batch: bool, skip_errors: bool, + unique_name: Option, ) -> Result<()> { let worker_index = self.scope.index(); @@ -4055,6 +4056,10 @@ impl> DataflowGraphInner } = callbacks; let wrapper_2 = wrapper.clone(); + let output_connector_id = self.connector_threads.len() - self.connector_monitors.len(); + let stats_name = unique_name.unwrap_or(format!("subscribe-{output_connector_id}")); + let mut stats = OutputConnectorStats::new(stats_name); + let output_columns = self .extract_columns(table_handle, column_paths)? .as_collection(); @@ -4088,6 +4093,7 @@ impl> DataflowGraphInner // i.e., we are called after every worker has finished processing callbacks from // the first inspect for this frontier. if let Err(frontier) = event { + stats.on_time_committed(frontier.first().copied().map(|t| t.0)); if worker_index == 0 && frontier.is_empty() { if let Some(on_end) = on_end.as_mut() { wrapper_2 @@ -4108,6 +4114,13 @@ impl> DataflowGraphInner ); } } + if let Ok((_, batches)) = event { + for batch in batches { + stats.on_batch_started(); + stats.on_batch_entries_written(batch.data.len()); + stats.on_batch_finished(); + } + } }) .probe_with(&self.output_probe); @@ -4845,6 +4858,7 @@ impl Graph for InnerDataflowGraph { _callbacks: SubscribeCallbacks, _skip_persisted_batch: bool, _skip_errors: bool, + _unique_name: Option, ) -> Result<()> { Err(Error::IoNotPossible) } @@ -5059,7 +5073,7 @@ impl Graph for InnerDataflowGraph { _grouping_columns_paths: Vec, _reduced_column_paths: Vec, _combine_fn: StatefulCombineFn, - _external_persistent_id: Option<&ExternalPersistentId>, + _unique_name: Option<&UniqueName>, _table_properties: Arc, ) -> Result { Err(Error::NotSupportedInIteration) @@ -5168,7 +5182,7 @@ impl Graph for InnerDataflowGraph { _commit_duration: Option, _parallel_readers: usize, _table_properties: Arc, - _external_persistent_id: Option<&ExternalPersistentId>, + _unique_name: Option<&UniqueName>, ) -> Result { Err(Error::IoNotPossible) } @@ -5179,6 +5193,7 @@ impl Graph for InnerDataflowGraph { mut _data_formatter: Box, _table_handle: TableHandle, _column_paths: Vec, + _unique_name: Option, ) -> Result<()> { Err(Error::IoNotPossible) } @@ -5435,6 +5450,7 @@ impl> Graph for OuterDataflo callbacks: SubscribeCallbacks, skip_persisted_batch: bool, skip_errors: bool, + unique_name: Option, ) -> Result<()> { self.0.borrow_mut().subscribe_table( table_handle, @@ -5442,6 +5458,7 @@ impl> Graph for OuterDataflo callbacks, skip_persisted_batch, skip_errors, + unique_name, ) } @@ -5680,7 +5697,7 @@ impl> Graph for OuterDataflo grouping_columns_paths: Vec, reduced_column_paths: Vec, combine_fn: StatefulCombineFn, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, table_properties: Arc, ) -> Result { self.0.borrow_mut().deduplicate( @@ -5688,7 +5705,7 @@ impl> Graph for OuterDataflo grouping_columns_paths, reduced_column_paths, combine_fn, - external_persistent_id, + unique_name, table_properties, ) } @@ -5798,7 +5815,7 @@ impl> Graph for OuterDataflo commit_duration: Option, parallel_readers: usize, table_properties: Arc, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, ) -> Result { self.0.borrow_mut().connector_table( reader, @@ -5806,7 +5823,7 @@ impl> Graph for OuterDataflo commit_duration, parallel_readers, table_properties, - external_persistent_id, + unique_name, ) } @@ -5816,10 +5833,15 @@ impl> Graph for OuterDataflo data_formatter: Box, table_handle: TableHandle, column_paths: Vec, + unique_name: Option, ) -> Result<()> { - self.0 - .borrow_mut() - .output_table(data_sink, data_formatter, table_handle, column_paths) + self.0.borrow_mut().output_table( + data_sink, + data_formatter, + table_handle, + column_paths, + unique_name, + ) } fn set_operator_properties(&self, operator_properties: OperatorProperties) -> Result<()> { diff --git a/src/engine/dataflow/persist.rs b/src/engine/dataflow/persist.rs index 33d54610..fe95f5d3 100644 --- a/src/engine/dataflow/persist.rs +++ b/src/engine/dataflow/persist.rs @@ -25,32 +25,32 @@ use crate::engine::dataflow::operators::MapWrapped; use crate::engine::dataflow::shard::Shard; use crate::engine::dataflow::{MaybeUpdate, Poller, SortingCell}; use crate::engine::reduce::IntSumState; -use crate::engine::{Error, Key, Result, Timestamp, Value}; +use crate::engine::{Key, Result, Timestamp, Value}; use crate::persistence::config::PersistenceManagerConfig; use crate::persistence::operator_snapshot::{OperatorSnapshotReader, OperatorSnapshotWriter}; use crate::persistence::tracker::{ RequiredPersistenceMode, SharedWorkerPersistentStorage, WorkerPersistentStorage, }; -use crate::persistence::{ExternalPersistentId, PersistenceTime, PersistentId}; +use crate::persistence::{PersistenceTime, PersistentId, UniqueName}; pub(super) fn effective_persistent_id( persistence_wrapper: &mut Box>, reader_is_internal: bool, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, required_persistence_mode: RequiredPersistenceMode, logic: impl FnOnce(u64) -> String, -) -> Result> +) -> Option where S: MaybeTotalScope, { let has_persistent_storage = persistence_wrapper .get_worker_persistent_storage() .is_some(); - if let Some(external_persistent_id) = external_persistent_id { + if let Some(unique_name) = unique_name { if has_persistent_storage { - Ok(Some(external_persistent_id.clone())) + Some(unique_name.clone()) } else { - Err(Error::NoPersistentStorage(external_persistent_id.clone())) + None } } else if has_persistent_storage && !reader_is_internal { let next_state_id = persistence_wrapper.next_state_id(); @@ -62,12 +62,12 @@ where if worker_persistent_storage.persistent_id_generation_enabled(required_persistence_mode) && worker_persistent_storage.table_persistence_enabled() { - Ok(Some(logic(next_state_id))) + Some(logic(next_state_id)) } else { - Ok(None) + None } } else { - Ok(None) + None } } diff --git a/src/engine/error.rs b/src/engine/error.rs index bdb5e788..fe50162e 100644 --- a/src/engine/error.rs +++ b/src/engine/error.rs @@ -10,7 +10,6 @@ use super::{Key, Value}; use crate::persistence::Error as PersistenceBackendError; use crate::connectors::data_storage::{ReadError, WriteError}; -use crate::persistence::ExternalPersistentId; #[allow(clippy::module_name_repetitions)] pub type DynError = Box; @@ -116,9 +115,6 @@ pub enum Error { trace: Trace, }, - #[error("persistent id {0} is assigned, but no persistent storage is configured")] - NoPersistentStorage(ExternalPersistentId), - #[error("snapshot writer failed: {0}")] SnapshotWriterError(#[source] WriteError), diff --git a/src/engine/graph.rs b/src/engine/graph.rs index 1e5c51d4..c8455fb9 100644 --- a/src/engine/graph.rs +++ b/src/engine/graph.rs @@ -20,7 +20,7 @@ use crate::connectors::data_format::{Formatter, Parser}; use crate::connectors::data_storage::{ReaderBuilder, Writer}; use crate::connectors::monitoring::ConnectorStats; use crate::external_integration::ExternalIndex; -use crate::persistence::ExternalPersistentId; +use crate::persistence::UniqueName; use crate::python_api::extract_value; use super::error::{DynResult, Trace}; @@ -737,6 +737,7 @@ pub trait Graph { callbacks: SubscribeCallbacks, skip_persisted_batch: bool, skip_errors: bool, + unique_name: Option, ) -> Result<()>; fn filter_table( @@ -877,7 +878,7 @@ pub trait Graph { grouping_columns_paths: Vec, reduced_column_paths: Vec, combine_fn: StatefulCombineFn, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, table_properties: Arc, ) -> Result; @@ -944,7 +945,7 @@ pub trait Graph { commit_duration: Option, parallel_readers: usize, table_properties: Arc, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, ) -> Result; fn output_table( @@ -953,6 +954,7 @@ pub trait Graph { data_formatter: Box, table_handle: TableHandle, column_paths: Vec, + unique_name: Option, ) -> Result<()>; fn set_operator_properties(&self, operator_properties: OperatorProperties) -> Result<()>; @@ -1191,6 +1193,7 @@ impl Graph for ScopedGraph { callbacks: SubscribeCallbacks, skip_persisted_batch: bool, skip_errors: bool, + unique_name: Option, ) -> Result<()> { self.try_with(|g| { g.subscribe_table( @@ -1199,6 +1202,7 @@ impl Graph for ScopedGraph { callbacks, skip_persisted_batch, skip_errors, + unique_name, ) }) } @@ -1446,7 +1450,7 @@ impl Graph for ScopedGraph { grouping_columns_paths: Vec, reduced_column_paths: Vec, combine_fn: StatefulCombineFn, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, table_properties: Arc, ) -> Result { self.try_with(|g| { @@ -1455,7 +1459,7 @@ impl Graph for ScopedGraph { grouping_columns_paths, reduced_column_paths, combine_fn, - external_persistent_id, + unique_name, table_properties, ) }) @@ -1555,7 +1559,7 @@ impl Graph for ScopedGraph { commit_duration: Option, parallel_readers: usize, table_properties: Arc, - external_persistent_id: Option<&ExternalPersistentId>, + unique_name: Option<&UniqueName>, ) -> Result { self.try_with(|g| { g.connector_table( @@ -1564,7 +1568,7 @@ impl Graph for ScopedGraph { commit_duration, parallel_readers, table_properties, - external_persistent_id, + unique_name, ) }) } @@ -1575,8 +1579,17 @@ impl Graph for ScopedGraph { data_formatter: Box, table_handle: TableHandle, column_paths: Vec, + unique_name: Option, ) -> Result<()> { - self.try_with(|g| g.output_table(data_sink, data_formatter, table_handle, column_paths)) + self.try_with(|g| { + g.output_table( + data_sink, + data_formatter, + table_handle, + column_paths, + unique_name, + ) + }) } fn set_operator_properties(&self, operator_properties: OperatorProperties) -> Result<()> { diff --git a/src/persistence/mod.rs b/src/persistence/mod.rs index a896e8d7..8c41ad7b 100644 --- a/src/persistence/mod.rs +++ b/src/persistence/mod.rs @@ -20,7 +20,7 @@ pub mod state; pub mod tracker; pub type PersistentId = u128; -pub type ExternalPersistentId = String; +pub type UniqueName = String; pub type SharedSnapshotWriter = Arc>; pub type SharedOperatorSnapshotWriter = Arc>>; @@ -31,7 +31,7 @@ pub trait IntoPersistentId { fn into_persistent_id(self) -> PersistentId; } -impl IntoPersistentId for ExternalPersistentId { +impl IntoPersistentId for UniqueName { fn into_persistent_id(self) -> PersistentId { let mut hasher = Hasher::default(); hasher.update(self.as_bytes()); diff --git a/src/python_api.rs b/src/python_api.rs index ad5a045c..f541c301 100644 --- a/src/python_api.rs +++ b/src/python_api.rs @@ -114,7 +114,7 @@ use crate::persistence::config::{ ConnectorWorkerPair, PersistenceManagerOuterConfig, PersistentStorageConfig, }; use crate::persistence::input_snapshot::Event as SnapshotEvent; -use crate::persistence::{ExternalPersistentId, IntoPersistentId, PersistentId}; +use crate::persistence::{IntoPersistentId, UniqueName}; use crate::pipe::{pipe, ReaderType, WriterType}; use crate::python_api::external_index_wrappers::PyExternalIndexFactory; use crate::timestamp::current_unix_timestamp_ms; @@ -696,7 +696,6 @@ impl From for PyErr { _ => ENGINE_ERROR_TYPE.bind(py).clone(), }, EngineError::IterationLimitTooSmall - | EngineError::NoPersistentStorage(_) | EngineError::InconsistentColumnProperties | EngineError::IdInTableProperties => PyValueError::type_object_bound(py), EngineError::ReaderFailed(ReadError::Py(e)) => return e, @@ -2275,28 +2274,35 @@ pub struct Scope { parent: Option>, license: Option, graph: SendWrapper, + is_persisted: bool, // empty_universe: Lazy>, universes: RefCell>>, columns: RefCell>>, tables: RefCell>>, error_logs: RefCell>>, - persistent_ids: RefCell>, + unique_names: RefCell>, event_loop: PyObject, total_connectors: RefCell, } impl Scope { - fn new(parent: Option>, event_loop: PyObject, license: Option) -> Self { + fn new( + parent: Option>, + event_loop: PyObject, + license: Option, + is_persisted: bool, + ) -> Self { Scope { parent, license, + is_persisted, graph: SendWrapper::new(ScopedGraph::new()), universes: RefCell::new(HashMap::new()), columns: RefCell::new(HashMap::new()), tables: RefCell::new(HashMap::new()), error_logs: RefCell::new(HashMap::new()), - persistent_ids: RefCell::new(HashSet::new()), + unique_names: RefCell::new(HashSet::new()), event_loop, total_connectors: RefCell::new(0), } @@ -2308,6 +2314,21 @@ impl Scope { self.tables.borrow_mut().clear(); self.error_logs.borrow_mut().clear(); } + + fn register_unique_name(&self, unique_name: Option<&UniqueName>) -> PyResult<()> { + if let Some(unique_name) = &unique_name { + let is_unique_id = self + .unique_names + .borrow_mut() + .insert((*unique_name).to_string()); + if !is_unique_id { + return Err(PyValueError::new_err(format!( + "Unique name '{unique_name}' used more than once" + ))); + } + } + Ok(()) + } } #[pymethods] @@ -2388,20 +2409,8 @@ impl Scope { ) -> PyResult> { let py = self_.py(); - let persistent_id = data_source.borrow().persistent_id.clone(); - if let Some(persistent_id) = &persistent_id { - let is_unique_id = self_ - .borrow() - .persistent_ids - .borrow_mut() - .insert(persistent_id.to_string()); - if !is_unique_id { - return Err(PyValueError::new_err(format!( - "Persistent ID '{persistent_id}' used more than once" - ))); - } - } - + let unique_name = properties.unique_name.clone(); + self_.borrow().register_unique_name(unique_name.as_ref())?; let connector_index = *self_.borrow().total_connectors.borrow(); *self_.borrow().total_connectors.borrow_mut() += 1; let (reader_impl, parallel_readers) = data_source.borrow().construct_reader( @@ -2410,6 +2419,7 @@ impl Scope { connector_index, self_.borrow().worker_index(), self_.borrow().license.as_ref(), + self_.borrow().is_persisted, )?; let parser_impl = data_format.borrow().construct_parser(py)?; @@ -2424,7 +2434,7 @@ impl Scope { .map(time::Duration::from_millis), parallel_readers, Arc::new(EngineTableProperties::flat(column_properties)), - persistent_id.as_ref(), + unique_name.as_ref(), )?; Table::new(self_, table_handle) } @@ -2454,6 +2464,7 @@ impl Scope { Some(self_.clone().unbind()), self_.borrow().event_loop.clone(), None, + false, ), )?; scope.borrow().graph.scoped(graph, || { @@ -2964,14 +2975,14 @@ impl Scope { Table::new(self_, table_handle) } - #[pyo3(signature = (table, grouping_columns_paths, reduced_column_paths, combine, persistent_id, table_properties))] + #[pyo3(signature = (table, grouping_columns_paths, reduced_column_paths, combine, unique_name, table_properties))] pub fn deduplicate( self_: &Bound, table: PyRef, #[pyo3(from_py_with = "from_py_iterable")] grouping_columns_paths: Vec, #[pyo3(from_py_with = "from_py_iterable")] reduced_column_paths: Vec, combine: Py, - persistent_id: Option, + unique_name: Option, table_properties: TableProperties, ) -> PyResult> { let table_handle = self_.borrow().graph.deduplicate( @@ -2979,7 +2990,7 @@ impl Scope { grouping_columns_paths, reduced_column_paths, wrap_stateful_combine(combine), - persistent_id.as_ref(), + unique_name.as_ref(), table_properties.0, )?; Table::new(self_, table_handle) @@ -3118,18 +3129,23 @@ impl Scope { #[pyo3(from_py_with = "from_py_iterable")] column_paths: Vec, data_sink: &Bound, data_format: &Bound, + unique_name: Option, ) -> PyResult<()> { let py = self_.py(); + self_.borrow().register_unique_name(unique_name.as_ref())?; let sink_impl = data_sink .borrow() .construct_writer(py, &data_format.borrow())?; let format_impl = data_format.borrow().construct_formatter(py)?; - self_ - .borrow() - .graph - .output_table(sink_impl, format_impl, table.handle, column_paths)?; + self_.borrow().graph.output_table( + sink_impl, + format_impl, + table.handle, + column_paths, + unique_name, + )?; Ok(()) } @@ -3144,7 +3160,9 @@ impl Scope { on_change: Py, on_time_end: Py, on_end: Py, + unique_name: Option, ) -> PyResult<()> { + self_.borrow().register_unique_name(unique_name.as_ref())?; let callbacks = SubscribeCallbacksBuilder::new() .wrapper(BatchWrapper::WithGil) .on_data(Box::new(move |key, values, time, diff| { @@ -3172,6 +3190,7 @@ impl Scope { callbacks, skip_persisted_batch, skip_errors, + unique_name, )?; Ok(()) } @@ -3280,7 +3299,7 @@ fn capture_table_data( Ok(()) })) .build(); - graph.subscribe_table(table.handle, column_paths, callbacks, false, false)?; + graph.subscribe_table(table.handle, column_paths, callbacks, false, false, None)?; } Ok(table_data) } @@ -3342,6 +3361,7 @@ pub fn run_with_new_graph( None } }; + let is_persisted = persistence_config.is_some(); let telemetry_config = EngineTelemetryConfig::create(&license, run_id, monitoring_server, trace_parent)?; let results: Vec> = run_with_wakeup_receiver(py, |wakeup_receiver| { @@ -3354,7 +3374,12 @@ pub fn run_with_new_graph( let captured_tables = Python::with_gil(|py| { let our_scope = &Bound::new( py, - Scope::new(None, event_loop.clone(), Some(scope_license.clone())), + Scope::new( + None, + event_loop.clone(), + Some(scope_license.clone()), + is_persisted, + ), )?; let tables: Vec<(PyRef
, Vec)> = our_scope.borrow().graph.scoped(graph, || { @@ -3722,10 +3747,10 @@ pub struct DataStorage { elasticsearch_params: Option>, parallel_readers: Option, python_subject: Option>, - persistent_id: Option, + unique_name: Option, max_batch_size: Option, object_pattern: String, - mock_events: Option>>, + mock_events: Option>>, table_name: Option, header_fields: Vec<(String, usize)>, key_field_index: Option, @@ -4038,7 +4063,7 @@ impl DataStorage { elasticsearch_params = None, parallel_readers = None, python_subject = None, - persistent_id = None, + unique_name = None, max_batch_size = None, object_pattern = "*".to_string(), mock_events = None, @@ -4067,10 +4092,10 @@ impl DataStorage { elasticsearch_params: Option>, parallel_readers: Option, python_subject: Option>, - persistent_id: Option, + unique_name: Option, max_batch_size: Option, object_pattern: String, - mock_events: Option>>, + mock_events: Option>>, table_name: Option, header_fields: Vec<(String, usize)>, key_field_index: Option, @@ -4095,7 +4120,7 @@ impl DataStorage { elasticsearch_params, parallel_readers, python_subject, - persistent_id, + unique_name, max_batch_size, object_pattern, mock_events, @@ -4402,33 +4427,31 @@ impl DataStorage { } } - fn internal_persistent_id(&self) -> Option { - self.persistent_id - .clone() - .map(IntoPersistentId::into_persistent_id) - } - - fn construct_fs_reader(&self) -> PyResult<(Box, usize)> { + fn construct_fs_reader(&self, is_persisted: bool) -> PyResult<(Box, usize)> { let storage = new_filesystem_reader( self.path()?, self.mode, - self.internal_persistent_id(), self.read_method, &self.object_pattern, + is_persisted, ) .map_err(|e| PyIOError::new_err(format!("Failed to initialize Filesystem reader: {e}")))?; Ok((Box::new(storage), 1)) } - fn construct_s3_reader(&self, py: pyo3::Python) -> PyResult<(Box, usize)> { + fn construct_s3_reader( + &self, + py: pyo3::Python, + is_persisted: bool, + ) -> PyResult<(Box, usize)> { let (_, deduced_path) = S3Scanner::deduce_bucket_and_path(self.path()?); let storage = new_s3_generic_reader( self.s3_bucket(py)?, deduced_path, self.mode, - self.internal_persistent_id(), self.read_method, self.downloader_threads_count()?, + is_persisted, ) .map_err(|e| PyRuntimeError::new_err(format!("Creating S3 reader failed: {e}")))?; Ok((Box::new(storage), 1)) @@ -4437,6 +4460,7 @@ impl DataStorage { fn construct_s3_csv_reader( &self, py: pyo3::Python, + is_persisted: bool, ) -> PyResult<(Box, usize)> { let (_, deduced_path) = S3Scanner::deduce_bucket_and_path(self.path()?); let storage = new_s3_csv_reader( @@ -4444,20 +4468,24 @@ impl DataStorage { deduced_path, self.build_csv_parser_settings(py), self.mode, - self.internal_persistent_id(), self.downloader_threads_count()?, + is_persisted, ) .map_err(|e| PyRuntimeError::new_err(format!("Creating S3 reader failed: {e}")))?; Ok((Box::new(storage), 1)) } - fn construct_csv_reader(&self, py: pyo3::Python) -> PyResult<(Box, usize)> { + fn construct_csv_reader( + &self, + py: pyo3::Python, + is_persisted: bool, + ) -> PyResult<(Box, usize)> { let reader = new_csv_filesystem_reader( self.path()?, self.build_csv_parser_settings(py), self.mode, - self.internal_persistent_id(), &self.object_pattern, + is_persisted, ) .map_err(|e| { PyIOError::new_err(format!("Failed to initialize CsvFilesystem reader: {e}")) @@ -4556,12 +4584,7 @@ impl DataStorage { seek_positions.insert(element.partition(), offset); } } - let reader = KafkaReader::new( - consumer, - topic.to_string(), - self.internal_persistent_id(), - seek_positions, - ); + let reader = KafkaReader::new(consumer, topic.to_string(), seek_positions); Ok((Box::new(reader), self.parallel_readers.unwrap_or(256))) } @@ -4574,17 +4597,13 @@ impl DataStorage { PyValueError::new_err("For Python connector, python_subject should be specified") })?; - if subject.borrow(py).is_internal && self.persistent_id.is_some() { + if subject.borrow(py).is_internal && self.unique_name.is_some() { return Err(PyValueError::new_err( - "Python connectors marked internal can't have persistent id", + "Python connectors marked internal can't have unique names", )); } - let reader = PythonReaderBuilder::new( - subject, - self.internal_persistent_id(), - data_format.value_fields_type_map(py), - ); + let reader = PythonReaderBuilder::new(subject, data_format.value_fields_type_map(py)); Ok((Box::new(reader), 1)) } @@ -4648,7 +4667,6 @@ impl DataStorage { self.delta_storage_options(py)?, data_format.value_fields_type_map(py), self.mode, - self.internal_persistent_id(), ) .map_err(|e| PyIOError::new_err(format!("Failed to connect to DeltaLake: {e}")))?; Ok((Box::new(reader), 1)) @@ -4668,19 +4686,14 @@ impl DataStorage { .await .map_err(|e| PyIOError::new_err(format!("Failed to connect to NATS: {e}")))?; let subscriber = client - .queue_subscribe(topic, consumer_queue) // Kafka "consumer group" equivalent to enable parallel reads + .queue_subscribe(topic.clone(), consumer_queue) // Kafka "consumer group" equivalent to enable parallel reads .await .map_err(|e| { PyIOError::new_err(format!("Failed to subscribe to NATS topic: {e}")) })?; Ok::(subscriber) })?; - let reader = NatsReader::new( - runtime, - subscriber, - worker_index, - self.internal_persistent_id(), - ); + let reader = NatsReader::new(runtime, subscriber, worker_index, topic); Ok((Box::new(reader), 32)) } @@ -4720,7 +4733,6 @@ impl DataStorage { &table_params, data_format.value_fields_type_map(py), self.mode, - self.internal_persistent_id(), ) .map_err(|e| { PyIOError::new_err(format!("Unable to start data lake input connector: {e}")) @@ -4736,12 +4748,13 @@ impl DataStorage { connector_index: usize, worker_index: usize, license: Option<&License>, + is_persisted: bool, ) -> PyResult<(Box, usize)> { match self.storage_type.as_ref() { - "fs" => self.construct_fs_reader(), - "s3" => self.construct_s3_reader(py), - "s3_csv" => self.construct_s3_csv_reader(py), - "csv" => self.construct_csv_reader(py), + "fs" => self.construct_fs_reader(is_persisted), + "s3" => self.construct_s3_reader(py, is_persisted), + "s3_csv" => self.construct_s3_csv_reader(py, is_persisted), + "csv" => self.construct_csv_reader(py, is_persisted), "kafka" => self.construct_kafka_reader(), "python" => self.construct_python_reader(py, data_format), "sqlite" => self.construct_sqlite_reader(py, data_format), @@ -4770,10 +4783,8 @@ impl DataStorage { } "mock" => { let mut events = HashMap::>::new(); - for ((external_persistent_id, worker_id), es) in self.mock_events.as_ref().unwrap() - { - let internal_persistent_id = - external_persistent_id.clone().into_persistent_id(); + for ((unique_name, worker_id), es) in self.mock_events.as_ref().unwrap() { + let internal_persistent_id = unique_name.clone().into_persistent_id(); events.insert((internal_persistent_id, *worker_id), es.clone()); } Ok(PersistentStorageConfig::Mock(events)) @@ -4806,7 +4817,7 @@ impl DataStorage { match file { Ok(f) => { let buf_writer = BufWriter::new(f); - FileWriter::new(buf_writer) + FileWriter::new(buf_writer, path.to_string()) } Err(_) => return Err(PyIOError::new_err("Filesystem operation (create) failed")), } @@ -5223,6 +5234,8 @@ pub struct ConnectorProperties { unsafe_trusted_ids: bool, #[pyo3(get)] column_properties: Vec, + #[pyo3(get)] + unique_name: Option, } #[pymethods] @@ -5231,17 +5244,20 @@ impl ConnectorProperties { #[pyo3(signature = ( commit_duration_ms = None, unsafe_trusted_ids = false, - column_properties = vec![] + column_properties = vec![], + unique_name = None ))] fn new( commit_duration_ms: Option, unsafe_trusted_ids: bool, #[pyo3(from_py_with = "from_py_iterable")] column_properties: Vec, + unique_name: Option, ) -> Self { Self { commit_duration_ms, unsafe_trusted_ids, column_properties, + unique_name, } } } diff --git a/tests/integration/helpers.rs b/tests/integration/helpers.rs index 1a17b8f5..f9c6c2b6 100644 --- a/tests/integration/helpers.rs +++ b/tests/integration/helpers.rs @@ -21,6 +21,7 @@ use pathway_engine::connectors::{Connector, Entry, PersistenceMode, SnapshotAcce use pathway_engine::engine::{Key, Timestamp, TotalFrontier, Value}; use pathway_engine::persistence::frontier::OffsetAntichain; use pathway_engine::persistence::input_snapshot::Event as SnapshotEvent; +use pathway_engine::persistence::PersistentId; #[derive(Debug)] pub struct FullReadResult { @@ -45,12 +46,11 @@ pub fn full_cycle_read( reader: Box, parser: &mut dyn Parser, persistent_storage: Option<&Arc>>, + persistent_id: Option, ) -> FullReadResult { - let maybe_persistent_id = reader.persistent_id(); - let offsets = Arc::new(Mutex::new(HashMap::::new())); if let Some(persistent_storage) = persistent_storage { - if let Some(persistent_id) = reader.persistent_id() { + if let Some(persistent_id) = persistent_id { persistent_storage .lock() .unwrap() @@ -60,14 +60,19 @@ pub fn full_cycle_read( let main_thread = thread::current(); let (sender, receiver) = mpsc::channel(); - let mut snapshot_writer = - Connector::snapshot_writer(reader.as_ref(), persistent_storage, SnapshotAccess::Full) - .unwrap(); + let mut snapshot_writer = Connector::snapshot_writer( + reader.as_ref(), + persistent_id, + persistent_storage, + SnapshotAccess::Full, + ) + .unwrap(); let mut reader = reader.build().expect("building the reader failed"); Connector::read_snapshot( &mut *reader, persistent_storage, + persistent_id, &sender, PersistenceMode::Batch, SnapshotAccess::Full, @@ -147,7 +152,7 @@ pub fn full_cycle_read( assert!(rewind_finish_sentinel_seen); - if maybe_persistent_id.is_some() && has_persistent_storage { + if persistent_id.is_some() && has_persistent_storage { let prev_finalized_time = persistent_storage .unwrap() .lock() diff --git a/tests/integration/test_arrow.rs b/tests/integration/test_arrow.rs index d037af47..cfc49a45 100644 --- a/tests/integration/test_arrow.rs +++ b/tests/integration/test_arrow.rs @@ -36,6 +36,10 @@ impl LakeBatchWriter for ArrowBatchWriter { utc_timezone_name: "UTC".into(), } } + + fn name(&self) -> String { + "test".to_string() + } } fn run_arrow_roadtrip(type_: Type, values: Vec) -> eyre::Result<()> { diff --git a/tests/integration/test_bytes.rs b/tests/integration/test_bytes.rs index 7c41ecf6..bbe94e05 100644 --- a/tests/integration/test_bytes.rs +++ b/tests/integration/test_bytes.rs @@ -13,7 +13,7 @@ use crate::helpers::ReplaceErrors; fn read_bytes_from_path(path: &str) -> eyre::Result> { let mut reader = - new_filesystem_reader(path, ConnectorMode::Static, None, ReadMethod::Full, "*")?; + new_filesystem_reader(path, ConnectorMode::Static, ReadMethod::Full, "*", false)?; let mut parser = IdentityParser::new( vec!["data".to_string()], false, diff --git a/tests/integration/test_connector_field_defaults.rs b/tests/integration/test_connector_field_defaults.rs index f13ccac9..959a7d7e 100644 --- a/tests/integration/test_connector_field_defaults.rs +++ b/tests/integration/test_connector_field_defaults.rs @@ -44,8 +44,8 @@ fn test_dsv_with_default_end_of_line() -> eyre::Result<()> { "tests/data/dsv_with_skips.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -98,8 +98,8 @@ fn test_dsv_with_default_middle_of_line() -> eyre::Result<()> { "tests/data/dsv_with_skips2.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -149,8 +149,8 @@ fn test_dsv_fails_without_default() -> eyre::Result<()> { "tests/data/dsv_with_skips.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -203,8 +203,8 @@ fn test_dsv_with_default_nullable() -> eyre::Result<()> { "tests/data/dsv_with_skips.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -257,9 +257,9 @@ fn test_jsonlines_fails_without_default() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = JsonLinesParser::new( Some(vec!["a".to_string()]), @@ -304,9 +304,9 @@ fn test_jsonlines_with_default() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines_with_skips.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = JsonLinesParser::new( Some(vec!["a".to_string()]), @@ -357,9 +357,9 @@ fn test_jsonlines_with_default_at_jsonpath() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines_with_skips.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = JsonLinesParser::new( Some(vec!["a".to_string()]), @@ -404,9 +404,9 @@ fn test_jsonlines_explicit_null_not_overridden() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines_with_skips_and_nulls.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = JsonLinesParser::new( Some(vec!["a".to_string()]), diff --git a/tests/integration/test_debezium.rs b/tests/integration/test_debezium.rs index 1beae9f8..ba2aff78 100644 --- a/tests/integration/test_debezium.rs +++ b/tests/integration/test_debezium.rs @@ -23,9 +23,9 @@ fn test_debezium_reads_ok() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/sample_debezium.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = DebeziumMessageParser::new( Some(vec!["id".to_string()]), @@ -173,9 +173,9 @@ fn test_debezium_mongodb_format() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/sample_debezium_mongodb.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = DebeziumMessageParser::new( Some(vec!["id".to_string()]), diff --git a/tests/integration/test_deltalake.rs b/tests/integration/test_deltalake.rs index b7319c77..c70bb203 100644 --- a/tests/integration/test_deltalake.rs +++ b/tests/integration/test_deltalake.rs @@ -104,7 +104,6 @@ fn read_with_connector(path: &str, type_: &Type) -> Result> { HashMap::new(), type_map, ConnectorMode::Static, - None, ) .unwrap(); let parser = diff --git a/tests/integration/test_dsv.rs b/tests/integration/test_dsv.rs index 2a6b50ac..155f681e 100644 --- a/tests/integration/test_dsv.rs +++ b/tests/integration/test_dsv.rs @@ -21,9 +21,9 @@ fn test_dsv_read_ok() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/sample.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new(Some(vec!["a".to_string()]), vec!["b".to_string()], ','), @@ -73,9 +73,9 @@ fn test_dsv_column_does_not_exist() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/sample.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new(Some(vec!["a".to_string()]), vec!["c".to_string()], ','), @@ -101,9 +101,9 @@ fn test_dsv_rows_parsing_ignore_type() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/sample_str_int.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new(Some(vec!["a".to_string()]), vec!["b".to_string()], ','), @@ -141,9 +141,9 @@ fn test_dsv_not_enough_columns() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/sample_bad_lines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new(Some(vec!["a".to_string()]), vec!["b".to_string()], ','), @@ -191,9 +191,9 @@ fn test_dsv_autogenerate_pkey() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/sample.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new(None, vec!["a".to_string(), "b".to_string()], ','), @@ -239,9 +239,9 @@ fn test_dsv_composite_pkey() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/sample_composite_pkey.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new( @@ -309,9 +309,9 @@ fn test_dsv_read_schema_ok() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/schema.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new( @@ -383,9 +383,9 @@ fn test_dsv_read_schema_nonparsable() -> eyre::Result<()> { let mut reader = new_filesystem_reader( "tests/data/incorrect_types.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut parser = DsvParser::new( DsvSettings::new( diff --git a/tests/integration/test_dsv_dir.rs b/tests/integration/test_dsv_dir.rs index fd034a67..42040af0 100644 --- a/tests/integration/test_dsv_dir.rs +++ b/tests/integration/test_dsv_dir.rs @@ -21,8 +21,8 @@ fn test_dsv_dir_ok() -> eyre::Result<()> { "tests/data/csvdir", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new(Some(vec!["key".to_string()]), vec!["foo".to_string()], ','), @@ -59,8 +59,8 @@ fn test_single_file_ok() -> eyre::Result<()> { "tests/data/sample.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new(Some(vec!["a".to_string()]), vec!["b".to_string()], ','), @@ -93,8 +93,8 @@ fn test_custom_delimiter() -> eyre::Result<()> { "tests/data/sql_injection.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -132,8 +132,8 @@ fn test_escape_fields() -> eyre::Result<()> { "tests/data/csv_fields_escaped.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -186,8 +186,8 @@ fn test_escape_newlines() -> eyre::Result<()> { "tests/data/csv_escaped_newlines.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( @@ -222,8 +222,8 @@ fn test_nonexistent_file() -> eyre::Result<()> { "tests/data/nonexistent_file.txt", builder, ConnectorMode::Static, - None, "*", + false, ); // We treat this path as a glob pattern, so the situation is normal: @@ -255,8 +255,8 @@ fn test_special_fields() -> eyre::Result<()> { "tests/data/csv_special_fields.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let parser = DsvParser::new( DsvSettings::new( diff --git a/tests/integration/test_jsonlines.rs b/tests/integration/test_jsonlines.rs index bd989793..58adcf4f 100644 --- a/tests/integration/test_jsonlines.rs +++ b/tests/integration/test_jsonlines.rs @@ -18,9 +18,9 @@ fn test_jsonlines_ok() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -63,9 +63,9 @@ fn test_jsonlines_incorrect_key() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -97,9 +97,9 @@ fn test_jsonlines_incomplete_key_to_null() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -127,9 +127,9 @@ fn test_jsonlines_incorrect_values() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -160,9 +160,9 @@ fn test_jsonlines_types_parsing() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines_types.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -254,9 +254,9 @@ fn test_jsonlines_complex_paths() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/json_complex_paths.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let mut routes = HashMap::new(); @@ -329,9 +329,9 @@ fn test_jsonlines_complex_paths_error() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/json_complex_paths.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ @@ -390,9 +390,9 @@ fn test_jsonlines_complex_path_ignore_errors() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/json_complex_paths.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ @@ -447,9 +447,9 @@ fn test_jsonlines_incorrect_key_verbose_error() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -484,9 +484,9 @@ fn test_jsonlines_incorrect_jsonpointer_verbose_error() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -518,9 +518,9 @@ fn test_jsonlines_failed_to_parse_field() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/json_complex_paths.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [("pet".to_string(), InnerSchemaField::new(Type::Any, None))]; let parser = JsonLinesParser::new( diff --git a/tests/integration/test_metadata.rs b/tests/integration/test_metadata.rs index 7fca623f..923bb9d4 100644 --- a/tests/integration/test_metadata.rs +++ b/tests/integration/test_metadata.rs @@ -33,9 +33,9 @@ fn test_metadata_fs_dir() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/csvdir/", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("key".to_string(), InnerSchemaField::new(Type::Int, None)), @@ -71,9 +71,9 @@ fn test_metadata_fs_file() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/minimal.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("key".to_string(), InnerSchemaField::new(Type::Int, None)), @@ -111,8 +111,8 @@ fn test_metadata_csv_dir() -> eyre::Result<()> { "tests/data/csvdir/", builder, ConnectorMode::Static, - None, "*", + false, )?; let schema = [ ("key".to_string(), InnerSchemaField::new(Type::Int, None)), @@ -152,8 +152,8 @@ fn test_metadata_csv_file() -> eyre::Result<()> { "tests/data/minimal.txt", builder, ConnectorMode::Static, - None, "*", + false, )?; let schema = [ ("key".to_string(), InnerSchemaField::new(Type::Int, None)), @@ -187,9 +187,9 @@ fn test_metadata_json_file() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -218,9 +218,9 @@ fn test_metadata_json_dir() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines/", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let schema = [ ("a".to_string(), InnerSchemaField::new(Type::String, None)), @@ -250,9 +250,9 @@ fn test_metadata_identity_file() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines.txt", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = IdentityParser::new( vec!["data".to_string(), "_metadata".to_string()], @@ -272,9 +272,9 @@ fn test_metadata_identity_dir() -> eyre::Result<()> { let reader = new_filesystem_reader( "tests/data/jsonlines/", ConnectorMode::Static, - None, ReadMethod::ByLine, "*", + false, )?; let parser = IdentityParser::new( vec!["data".to_string(), "_metadata".to_string()], diff --git a/tests/integration/test_seek.rs b/tests/integration/test_seek.rs index 47128762..2cfcb71c 100644 --- a/tests/integration/test_seek.rs +++ b/tests/integration/test_seek.rs @@ -18,6 +18,7 @@ use pathway_engine::connectors::data_storage::{ use pathway_engine::connectors::SessionType; use pathway_engine::engine::{Result, Type, Value}; use pathway_engine::persistence::tracker::WorkerPersistentStorage; +use pathway_engine::persistence::PersistentId; enum TestedFormat { Csv, @@ -28,8 +29,7 @@ fn csv_reader_parser_pair(input_path: &str) -> Result<(Box, B let mut builder = csv::ReaderBuilder::new(); builder.has_headers(false); let reader = - new_csv_filesystem_reader(input_path, builder, ConnectorMode::Static, Some(1), "*") - .unwrap(); + new_csv_filesystem_reader(input_path, builder, ConnectorMode::Static, "*", true).unwrap(); let schema = [ ("key".to_string(), InnerSchemaField::new(Type::String, None)), ( @@ -52,9 +52,9 @@ fn json_reader_parser_pair(input_path: &str) -> Result<(Box, let reader = new_filesystem_reader( input_path, ConnectorMode::Static, - Some(1), ReadMethod::ByLine, "*", + true, ) .unwrap(); let schema = [ @@ -79,12 +79,18 @@ fn full_cycle_read_kv( format: TestedFormat, input_path: &Path, persistent_storage: Option<&Arc>>, + persistent_id: Option, ) -> Result { let (reader, mut parser) = match format { TestedFormat::Csv => csv_reader_parser_pair(input_path.to_str().unwrap())?, TestedFormat::Json => json_reader_parser_pair(input_path.to_str().unwrap())?, }; - Ok(full_cycle_read(reader, parser.as_mut(), persistent_storage)) + Ok(full_cycle_read( + reader, + parser.as_mut(), + persistent_storage, + persistent_id, + )) } #[test] @@ -98,7 +104,8 @@ fn test_csv_file_recovery() -> eyre::Result<()> { std::fs::write(&input_path, "key,value\n1,2\na,b").unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, true); - let data_stream = full_cycle_read_kv(TestedFormat::Csv, &input_path, Some(&tracker))?; + let data_stream = + full_cycle_read_kv(TestedFormat::Csv, &input_path, Some(&tracker), Some(1))?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -117,7 +124,8 @@ fn test_csv_file_recovery() -> eyre::Result<()> { std::fs::write(&input_path, "key,value\n1,2\na,b\nc,d\n55,66").unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, false); - let data_stream = full_cycle_read_kv(TestedFormat::Csv, &input_path, Some(&tracker))?; + let data_stream = + full_cycle_read_kv(TestedFormat::Csv, &input_path, Some(&tracker), Some(1))?; eprintln!("data stream after: {:?}", data_stream.new_parsed_entries); assert_eq!( data_stream.new_parsed_entries, @@ -171,7 +179,8 @@ fn test_csv_dir_recovery() -> eyre::Result<()> { { let tracker = create_persistence_manager(&pstorage_root_path, true); - let data_stream = full_cycle_read_kv(TestedFormat::Csv, &inputs_dir_path, Some(&tracker))?; + let data_stream = + full_cycle_read_kv(TestedFormat::Csv, &inputs_dir_path, Some(&tracker), Some(1))?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -207,7 +216,8 @@ fn test_csv_dir_recovery() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, false); - let data_stream = full_cycle_read_kv(TestedFormat::Csv, &inputs_dir_path, Some(&tracker))?; + let data_stream = + full_cycle_read_kv(TestedFormat::Csv, &inputs_dir_path, Some(&tracker), Some(1))?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -262,7 +272,8 @@ fn test_json_file_recovery() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, true); - let data_stream = full_cycle_read_kv(TestedFormat::Json, &input_path, Some(&tracker))?; + let data_stream = + full_cycle_read_kv(TestedFormat::Json, &input_path, Some(&tracker), Some(1))?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -281,7 +292,8 @@ fn test_json_file_recovery() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, false); - let data_stream = full_cycle_read_kv(TestedFormat::Json, &input_path, Some(&tracker))?; + let data_stream = + full_cycle_read_kv(TestedFormat::Json, &input_path, Some(&tracker), Some(1))?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -320,7 +332,12 @@ fn test_json_folder_recovery() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, true); - let data_stream = full_cycle_read_kv(TestedFormat::Json, &inputs_dir_path, Some(&tracker))?; + let data_stream = full_cycle_read_kv( + TestedFormat::Json, + &inputs_dir_path, + Some(&tracker), + Some(1), + )?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -346,7 +363,12 @@ fn test_json_folder_recovery() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, false); - let data_stream = full_cycle_read_kv(TestedFormat::Json, &inputs_dir_path, Some(&tracker))?; + let data_stream = full_cycle_read_kv( + TestedFormat::Json, + &inputs_dir_path, + Some(&tracker), + Some(1), + )?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -387,7 +409,12 @@ fn test_json_recovery_with_new_file() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, true); - let data_stream = full_cycle_read_kv(TestedFormat::Json, &inputs_dir_path, Some(&tracker))?; + let data_stream = full_cycle_read_kv( + TestedFormat::Json, + &inputs_dir_path, + Some(&tracker), + Some(1), + )?; assert_eq!( data_stream.new_parsed_entries, vec![ @@ -408,7 +435,12 @@ fn test_json_recovery_with_new_file() -> eyre::Result<()> { .unwrap(); { let tracker = create_persistence_manager(&pstorage_root_path, false); - let data_stream = full_cycle_read_kv(TestedFormat::Json, &inputs_dir_path, Some(&tracker))?; + let data_stream = full_cycle_read_kv( + TestedFormat::Json, + &inputs_dir_path, + Some(&tracker), + Some(1), + )?; assert_eq!( data_stream.new_parsed_entries, vec![