From 2fccbdb849d8b1cfc243e9b0e974456bdd0c0906 Mon Sep 17 00:00:00 2001 From: Will Jones Date: Wed, 26 Jul 2023 10:19:41 -0700 Subject: [PATCH 01/99] Start python api docs --- docs/python_api.md | 33 ++++++++++++ docs/requirements.txt | 3 ++ docs/usage/examining-table.md | 23 ++++----- docs/usage/index.md | 2 +- docs/usage/loading-table.md | 14 ++---- mkdocs.yml | 24 ++++++++- python/deltalake/_internal.pyi | 92 ++++++++++------------------------ python/src/filesystem.rs | 6 +-- python/src/lib.rs | 4 +- python/src/schema.rs | 12 ++--- 10 files changed, 112 insertions(+), 101 deletions(-) create mode 100644 docs/python_api.md create mode 100644 docs/requirements.txt diff --git a/docs/python_api.md b/docs/python_api.md new file mode 100644 index 0000000000..4837122c43 --- /dev/null +++ b/docs/python_api.md @@ -0,0 +1,33 @@ +# Python API Reference + +## DeltaTable + +::: deltalake.table + +## Writing Delta Tables + +::: deltalake.write_deltalake + +## Delta Lake Schemas + +Schemas, fields, and data types are provided in the ``deltalake.schema`` submodule. + +::: deltalake.schema.Schema + +::: deltalake.schema.PrimitiveType + +::: deltalake.schema.ArrayType + +::: deltalake.schema.MapType + +::: deltalake.schema.Field + +::: deltalake.schema.StructType + +## Data Catalog + +::: deltalake.data_catalog + +## Delta Storage Handler + +::: deltalake.fs diff --git a/docs/requirements.txt b/docs/requirements.txt new file mode 100644 index 0000000000..608b198e25 --- /dev/null +++ b/docs/requirements.txt @@ -0,0 +1,3 @@ +mkdocs +mkdocstrings[python] +mkdocs-autorefs \ No newline at end of file diff --git a/docs/usage/examining-table.md b/docs/usage/examining-table.md index 5641a926b3..c4cbfb0836 100644 --- a/docs/usage/examining-table.md +++ b/docs/usage/examining-table.md @@ -14,7 +14,7 @@ The delta log maintains basic metadata about a table, including: to have data deleted from it. Get metadata from a table with the -`DeltaTable.metadata` method: +[DeltaTable.metadata()][] method: ``` python >>> from deltalake import DeltaTable @@ -27,12 +27,12 @@ Metadata(id: 5fba94ed-9794-4965-ba6e-6ee3c0d22af9, name: None, description: None The schema for the table is also saved in the transaction log. It can either be retrieved in the Delta Lake form as -`deltalake.schema.Schema` or as a +[deltalake.schema.Schema][] or as a PyArrow schema. The first allows you to introspect any column-level metadata stored in the schema, while the latter represents the schema the table will be loaded into. -Use `DeltaTable.schema` to retrieve the delta lake schema: +Use [DeltaTable.schema][] to retrieve the delta lake schema: ``` python >>> from deltalake import DeltaTable @@ -43,14 +43,14 @@ Schema([Field(id, PrimitiveType("long"), nullable=True)]) These schemas have a JSON representation that can be retrieved. To reconstruct from json, use -`deltalake.schema.Schema.from_json()`. +[deltalake.schema.Schema.from_json()][]. ``` python >>> dt.schema().json() '{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,"metadata":{}}]}' ``` -Use `deltalake.schema.Schema.to_pyarrow()` to retrieve the PyArrow schema: +Use [deltalake.schema.Schema.to_pyarrow()][] to retrieve the PyArrow schema: ``` python >>> dt.schema().to_pyarrow() @@ -65,15 +65,12 @@ table, when, and by whom. This information is retained for 30 days by default, unless otherwise specified by the table configuration `delta.logRetentionDuration`. -::: note -::: title -Note -::: +!!! note + + This information is not written by all writers and different writers may + use different schemas to encode the actions. For Spark\'s format, see: + -This information is not written by all writers and different writers may -use different schemas to encode the actions. For Spark\'s format, see: - -::: To view the available history, use `DeltaTable.history`: diff --git a/docs/usage/index.md b/docs/usage/index.md index c765f7b5df..5f9624653a 100644 --- a/docs/usage/index.md +++ b/docs/usage/index.md @@ -1,6 +1,6 @@ # Usage -A `DeltaTable` represents the state of a +A [DeltaTable][] represents the state of a delta table at a particular version. This includes which files are currently part of the table, the schema of the table, and other metadata such as creation time. diff --git a/docs/usage/loading-table.md b/docs/usage/loading-table.md index 26d78a593f..1af7df95e5 100644 --- a/docs/usage/loading-table.md +++ b/docs/usage/loading-table.md @@ -109,12 +109,8 @@ version number or datetime string: >>> dt.load_with_datetime("2021-11-04 00:05:23.283+00:00") ``` -::: warning -::: title -Warning -::: - -Previous table versions may not exist if they have been vacuumed, in -which case an exception will be thrown. See [Vacuuming -tables](#vacuuming-tables) for more information. -::: \ No newline at end of file +!!! warning + + Previous table versions may not exist if they have been vacuumed, in + which case an exception will be thrown. See [Vacuuming + tables](#vacuuming-tables) for more information. diff --git a/mkdocs.yml b/mkdocs.yml index b6cf710863..dd25578b4e 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -10,4 +10,26 @@ nav: - Examining a Delta Table: usage/examining-table.md - Querying a Delta Table: usage/querying-delta-tables.md - Managing a Delta Table: usage/managing-tables.md - - Writing Delta Tables: usage/writing-delta-tables.md \ No newline at end of file + - Writing Delta Tables: usage/writing-delta-tables.md + - API Reference: python_api.md + +plugins: +- autorefs +- mkdocstrings: + handlers: + python: + path: [../python] + rendering: + heading_level: 4 + show_source: false + show_symbol_type_in_heading: true + show_signature_annotations: true + show_root_heading: true + members_order: source + import: + # for cross references + - https://arrow.apache.org/docs/objects.inv + - https://pandas.pydata.org/docs/objects.inv + +markdown_extensions: + - admonition \ No newline at end of file diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 48da1d47df..5ba3b2b077 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -13,23 +13,22 @@ from deltalake.writer import AddAction __version__: str -RawDeltaTable: Any -rust_core_version: Callable[[], str] - -write_new_deltalake: Callable[ - [ - str, - pa.Schema, - List[AddAction], - str, - List[str], - Optional[str], - Optional[str], - Optional[Mapping[str, Optional[str]]], - Optional[Dict[str, str]], - ], - None, -] +class RawDeltaTable: + ... + +def rust_core_version() -> str: ... + +def write_new_deltalake( + table_uri: str, + schema: pa.Schema, + add_actions: List[AddAction], + _mode: str, + partition_by: List[str], + name: Optional[str], + description: Optional[str], + configuration: Optional[Mapping[str, Optional[str]]], + storage_options: Optional[Dict[str, str]], +): ... def batch_distinct(batch: pa.RecordBatch) -> pa.RecordBatch: ... @@ -93,34 +92,21 @@ class Field: *, nullable: bool = True, metadata: Optional[Dict[str, Any]] = None, - ) -> None: - """A named field, with a data type, nullability, and optional metadata.""" + ) -> None: ... name: str - """The field name.""" type: DataType - """The field data type.""" nullable: bool - """The field nullability.""" metadata: Dict[str, Any] - """The field metadata.""" - def to_json(self) -> str: - """Get the JSON representation of the Field. + def to_json(self) -> str: ... - :rtype: str - """ @staticmethod - def from_json(json: str) -> "Field": - """Create a new Field from a JSON string. + def from_json(json: str) -> "Field": ... + + def to_pyarrow(self) -> pa.Field: ... - :param json: A json string representing the Field. - :rtype: Field - """ - def to_pyarrow(self) -> pa.Field: - """Convert field to a pyarrow.Field.""" @staticmethod - def from_pyarrow(type: pa.Field) -> "Field": - """Create a new field from pyarrow.Field.""" + def from_pyarrow(type: pa.Field) -> "Field": ... class StructType: def __init__(self, fields: List[Field]) -> None: ... @@ -138,41 +124,15 @@ class Schema: def __init__(self, fields: List[Field]) -> None: ... fields: List[Field] invariants: List[Tuple[str, str]] - """The list of invariants defined on the table. - - The first string in each tuple is the field path, the second is the SQL of the invariant. - """ - def to_json(self) -> str: - """Get the JSON representation of the schema. + def to_json(self) -> str: ... - :rtype: str - """ @staticmethod - def from_json(json: str) -> "Schema": - """Create a new Schema from a JSON string. - - :param schema_json: a JSON string - :rtype: Schema - """ - def to_pyarrow(self, as_large_types: bool = False) -> pa.Schema: - """Return equivalent PyArrow schema. - - Note: this conversion is lossy as the Invariants are not stored in pyarrow.Schema. + def from_json(json: str) -> "Schema": ... + def to_pyarrow(self, as_large_types: bool = False) -> pa.Schema: ... - :param as_large_types: get schema with all variable size types (list, - binary, string) as large variants (with int64 indices). This is for - compatibility with systems like Polars that only support the large - versions of Arrow types. - :rtype: pyarrow.Schema - """ @staticmethod - def from_pyarrow(type: pa.Schema) -> "Schema": - """Create a new Schema from a pyarrow.Schema. - - :param data_type: a PyArrow schema - :rtype: Schema - """ + def from_pyarrow(type: pa.Schema) -> "Schema": ... class ObjectInputFile: @property diff --git a/python/src/filesystem.rs b/python/src/filesystem.rs index 176045cdcc..d7387ce099 100644 --- a/python/src/filesystem.rs +++ b/python/src/filesystem.rs @@ -19,7 +19,7 @@ pub(crate) struct FsConfig { pub(crate) options: HashMap, } -#[pyclass(subclass)] +#[pyclass(subclass, module = "deltalake._internal")] #[derive(Debug, Clone)] pub struct DeltaFileSystemHandler { pub(crate) inner: Arc, @@ -265,7 +265,7 @@ impl DeltaFileSystemHandler { // TODO the C++ implementation track an internal lock on all random access files, DO we need this here? // TODO add buffer to store data ... -#[pyclass(weakref)] +#[pyclass(weakref, module = "deltalake._internal")] #[derive(Debug, Clone)] pub struct ObjectInputFile { store: Arc, @@ -433,7 +433,7 @@ impl ObjectInputFile { // TODO the C++ implementation track an internal lock on all random access files, DO we need this here? // TODO add buffer to store data ... -#[pyclass(weakref)] +#[pyclass(weakref, module = "deltalake._internal")] pub struct ObjectOutputStream { store: Arc, rt: Arc, diff --git a/python/src/lib.rs b/python/src/lib.rs index a78978cff1..a5472114ee 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -49,7 +49,7 @@ enum PartitionFilterValue<'a> { Multiple(Vec<&'a str>), } -#[pyclass] +#[pyclass(module = "deltalake._internal")] struct RawDeltaTable { _table: deltalake::DeltaTable, // storing the config additionally on the table helps us make pickling work. @@ -807,7 +807,7 @@ fn write_new_deltalake( Ok(()) } -#[pyclass(name = "DeltaDataChecker")] +#[pyclass(name = "DeltaDataChecker", module = "deltalake._internal")] struct PyDeltaDataChecker { inner: DeltaDataChecker, rt: tokio::runtime::Runtime, diff --git a/python/src/schema.rs b/python/src/schema.rs index 43d386a617..425d775a9e 100644 --- a/python/src/schema.rs +++ b/python/src/schema.rs @@ -113,7 +113,7 @@ fn python_type_to_schema(ob: PyObject, py: Python) -> PyResult { /// * "decimal(, )" /// /// :param data_type: string representation of the data type -#[pyclass(module = "deltalake.schema")] +#[pyclass(module = "deltalake._internal")] #[derive(Clone)] pub struct PrimitiveType { inner_type: String, @@ -247,7 +247,7 @@ impl PrimitiveType { /// ArrayType(PrimitiveType("integer"), contains_null=True) /// >>> ArrayType("integer", contains_null=False) /// ArrayType(PrimitiveType("integer"), contains_null=False) -#[pyclass(module = "deltalake.schema")] +#[pyclass(module = "deltalake._internal")] #[derive(Clone)] pub struct ArrayType { inner_type: SchemaTypeArray, @@ -409,7 +409,7 @@ impl ArrayType { /// MapType(PrimitiveType("integer"), PrimitiveType("string"), value_contains_null=True) /// >>> MapType("integer", "string", value_contains_null=False) /// MapType(PrimitiveType("integer"), PrimitiveType("string"), value_contains_null=False) -#[pyclass(module = "deltalake.schema")] +#[pyclass(module = "deltalake._internal")] #[derive(Clone)] pub struct MapType { inner_type: SchemaTypeMap, @@ -592,7 +592,7 @@ impl MapType { /// /// >>> Field("my_col", "integer", metadata={"custom_metadata": {"test": 2}}) /// Field("my_col", PrimitiveType("integer"), nullable=True, metadata={"custom_metadata": {"test": 2}}) -#[pyclass(module = "deltalake.schema")] +#[pyclass(module = "deltalake._internal")] #[derive(Clone)] pub struct Field { inner: SchemaField, @@ -770,7 +770,7 @@ impl Field { /// /// >>> StructType([Field("x", "integer"), Field("y", "string")]) /// StructType([Field(x, PrimitiveType("integer"), nullable=True), Field(y, PrimitiveType("string"), nullable=True)]) -#[pyclass(subclass, module = "deltalake.schema")] +#[pyclass(subclass, module = "deltalake._internal")] #[derive(Clone)] pub struct StructType { inner_type: SchemaTypeStruct, @@ -943,7 +943,7 @@ pub fn schema_to_pyobject(schema: &Schema, py: Python) -> PyResult { /// >>> import pyarrow as pa /// >>> Schema.from_pyarrow(pa.schema({"x": pa.int32(), "y": pa.string()})) /// Schema([Field(x, PrimitiveType("integer"), nullable=True), Field(y, PrimitiveType("string"), nullable=True)]) -#[pyclass(extends = StructType, name = "Schema", module = "deltalake.schema")] +#[pyclass(extends = StructType, name = "Schema", module = "deltalake._internal")] pub struct PySchema; #[pymethods] From 81d5a999f91ae9714262065d2e4430916aa8623a Mon Sep 17 00:00:00 2001 From: "R. Tyler Croy" Date: Tue, 19 Sep 2023 19:21:44 -0700 Subject: [PATCH 02/99] Proposed updated CODEOWNERS to allow better review notifications Based on current pull request feedback and maintenance trends I'm suggesting these rules to get the right people on the reviews by default. Closes #1553 --- .github/CODEOWNERS | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 480bc5c521..274e1e161b 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,4 +1,6 @@ -* @houqp @xianwill @wjones127 @fvaleye @roeap @rtyler @mosyp +rust/ @wjones127 @roeap @rtyler proofs/ @houqp -python/ @wjones127 @fvaleye @rtyler @roeap @houqp +python/ @wjones127 @fvaleye @roeap tlaplus/ @houqp +.github/ @wjones127 @rtyler +docs/ @MrPowers From f6cb3ca60fc2d535de1e1aed5f3574bb3ff33599 Mon Sep 17 00:00:00 2001 From: "R. Tyler Croy" Date: Tue, 19 Sep 2023 20:18:29 -0700 Subject: [PATCH 03/99] Compensate for invalid log files created by Delta Live Tables It would appear that in some cases Delta Live Tables will create a Delta table which does not adhere to the Delta Table protocol. The metaData action as a **required** `schemaString` property which simply doesn't exist. Since it appears that this only exists at version zero of the transaction log, and the _actual_ schema exists in the following versions of the table (e.g. 1), this change introduces a default deserializer on the MetaData action which provides a simple empty schema. This is an alternative implementation to #1305 which is a bit more invasive and makes our schema_string struct member `Option` which I do not believe is worth it for this unfortunate compatibility issue Closes #1305, #1302, #1357 Sponsored-by: Databricks Inc --- rust/src/action/mod.rs | 7 +++++++ .../_delta_log/00000000000000000000.json | 3 +++ .../_delta_log/00000000000000000001.json | 3 +++ rust/tests/read_delta_log_test.rs | 10 ++++++++++ 4 files changed, 23 insertions(+) create mode 100644 rust/tests/data/delta-live-table/_delta_log/00000000000000000000.json create mode 100644 rust/tests/data/delta-live-table/_delta_log/00000000000000000001.json diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index 5547129550..d411a502cd 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -506,6 +506,12 @@ impl Default for Format { } } +/// Return a default empty schema to be used for edge-cases when a schema is missing +fn default_schema() -> String { + warn!("A `metaData` action was missing a `schemaString` and has been given an empty schema"); + r#"{"type":"struct", "fields": []}"#.into() +} + /// Action that describes the metadata of the table. /// This is a top-level action in Delta log entries. #[derive(Serialize, Deserialize, Debug, Default, Clone)] @@ -520,6 +526,7 @@ pub struct MetaData { /// Specification of the encoding for the files stored in the table pub format: Format, /// Schema of the table + #[serde(default = "default_schema")] pub schema_string: String, /// An array containing the names of columns by which the data should be partitioned pub partition_columns: Vec, diff --git a/rust/tests/data/delta-live-table/_delta_log/00000000000000000000.json b/rust/tests/data/delta-live-table/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..14a083ee45 --- /dev/null +++ b/rust/tests/data/delta-live-table/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1681798578373,"operation":"DLT REFRESH","operationParameters":{"pipelineId":"5c7b47e9-12d6-4986-a601-6716734281ce","updateId":"adf780f9-789f-4857-8d41-e4d9938b61d5"},"isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Databricks-Runtime/dlt:11.3-delta-pipelines-801d604-ff1aff6-f0f113d-custom-local","txnId":"a2ad05c8-4559-41bc-b7b1-a94773bd2286"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"ac0a0120-970e-4d8c-ae92-b5244b055d6e","format":{"provider":"parquet","options":{}},"partitionColumns":[],"configuration":{},"createdTime":1681798577757}} \ No newline at end of file diff --git a/rust/tests/data/delta-live-table/_delta_log/00000000000000000001.json b/rust/tests/data/delta-live-table/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..ed4bd9056a --- /dev/null +++ b/rust/tests/data/delta-live-table/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1681798612850,"operation":"DLT SETUP","operationParameters":{"pipelineId":"5c7b47e9-12d6-4986-a601-6716734281ce","updateId":"adf780f9-789f-4857-8d41-e4d9938b61d5"},"readVersion":0,"isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Databricks-Runtime/dlt:11.3-delta-pipelines-801d604-ff1aff6-f0f113d-custom-local","txnId":"c1e3c149-d6b1-4a51-b7a6-89c328d14833"}} +{"protocol":{"minReaderVersion":2,"minWriterVersion":5}} +{"metaData":{"id":"ac0a0120-970e-4d8c-ae92-b5244b055d6e","name":"SnowflakeTest_Snowflake_DTL_SHERPA_USER_TABLE","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"sherpa_user_id\",\"type\":\"decimal(38,0)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"enabled\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_login\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"first_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"last_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"full_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"email\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"job_title\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"hire_date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"skypoint_delta_index\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"pipelines.pipelineId":"5c7b47e9-12d6-4986-a601-6716734281ce","pipelines.autoOptimize.managed":"false","pipelines.metastore.tableName":"automation_retailsandbox.SnowflakeTest_Snowflake_DTL_SHERPA_USER_TABLE"},"createdTime":1681798577757}} \ No newline at end of file diff --git a/rust/tests/read_delta_log_test.rs b/rust/tests/read_delta_log_test.rs index e8a96644b0..7167ea3c44 100644 --- a/rust/tests/read_delta_log_test.rs +++ b/rust/tests/read_delta_log_test.rs @@ -149,3 +149,13 @@ async fn test_read_liquid_table() -> DeltaResult<()> { let _table = deltalake::open_table(&path).await?; Ok(()) } + +// test for: https://github.com/delta-io/delta-rs/issues/1302 +#[tokio::test] +async fn read_delta_table_from_dlt() { + let table = deltalake::open_table("./tests/data/delta-live-table") + .await + .unwrap(); + assert_eq!(table.version(), 1); + assert!(table.schema().is_some()); +} From 9ef52a84f78bce09a93f2e72f829b071150c2823 Mon Sep 17 00:00:00 2001 From: "R. Tyler Croy" Date: Tue, 19 Sep 2023 21:59:03 -0700 Subject: [PATCH 04/99] chore: fix the incorrect Slack link in our readme not sure what the deal with the go.delta.io service, no idea where that lives Fixes #1636 --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 09d8fa4753..d75681de12 100644 --- a/README.md +++ b/README.md @@ -32,7 +32,7 @@ Deltalake - + #delta-rs in the Delta Lake Slack workspace

@@ -106,7 +106,7 @@ You can also try Delta Lake docker at [DockerHub](https://go.delta.io/dockerhub) We encourage you to reach out, and are [commited](https://github.com/delta-io/delta-rs/blob/main/CODE_OF_CONDUCT.md) to provide a welcoming community. -- [Join us in our Slack workspace](https://go.delta.io/slack) +- [Join us in our Slack workspace](https://join.slack.com/t/delta-users/shared_invite/zt-23h0xwez7-wDTm43ZVEW2ZcbKn6Bc8Fg) - [Report an issue](https://github.com/delta-io/delta-rs/issues/new?template=bug_report.md) - Looking to contribute? See our [good first issues](https://github.com/delta-io/delta-rs/contribute). From d151a974e4d5772067f0153f3c0843e1f955b6bc Mon Sep 17 00:00:00 2001 From: Eero Lihavainen Date: Thu, 21 Sep 2023 15:03:18 +0300 Subject: [PATCH 05/99] enable offset listing for s3 --- rust/src/storage/s3.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/rust/src/storage/s3.rs b/rust/src/storage/s3.rs index b7da2ae4d7..ec5e6a344a 100644 --- a/rust/src/storage/s3.rs +++ b/rust/src/storage/s3.rs @@ -474,6 +474,14 @@ impl ObjectStore for S3StorageBackend { self.inner.list(prefix).await } + async fn list_with_offset( + &self, + prefix: Option<&Path>, + offset: &Path, + ) -> ObjectStoreResult>> { + self.inner.list_with_offset(prefix, offset).await + } + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { self.inner.list_with_delimiter(prefix).await } From 312a7cbcb21f74d3e59a2de7c364374764b8a217 Mon Sep 17 00:00:00 2001 From: Simon Vandel Sillesen Date: Fri, 22 Sep 2023 07:31:40 +0100 Subject: [PATCH 06/99] Make docs.rs build docs with all features enabled I was confused that I could not find the documentation integrating datafusion with delta-rs. With this PR, everything should show up. Perhaps docs for a feature gated method should also mention which feature is required. Similar to what Tokio does. Perhaps it could be done in followup PRs. --- rust/Cargo.toml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 72ad44bbac..90941f8fe8 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -12,6 +12,9 @@ repository = "https://github.com/delta-io/delta.rs" readme = "README.md" edition = "2021" +[package.metadata.docs.rs] +all-features = true + [dependencies] # arrow arrow = { workspace = true, optional = true } From 72f8531c838b9ccbf9600862fdb61a32917377a2 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris Date: Fri, 22 Sep 2023 23:55:41 +0200 Subject: [PATCH 07/99] feat: expose min_commit_interval to `optimize.compact` and `optimize.z_order` (#1645) # Description Exposes min_commit_interval in the Python API to `optimize.compact` and `optimize.z_order`. Added one test-case to verify the min_commit_interval. # Related Issue(s) closes #1640 --------- Co-authored-by: Will Jones --- python/deltalake/table.py | 37 +++++++++++++++++++++++++++++++++-- python/docs/source/conf.py | 1 + python/src/lib.rs | 14 +++++++++++-- python/tests/test_optimize.py | 22 +++++++++++++++++++++ 4 files changed, 70 insertions(+), 4 deletions(-) diff --git a/python/deltalake/table.py b/python/deltalake/table.py index cf7d844e11..367debbf18 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -2,7 +2,7 @@ import operator import warnings from dataclasses import dataclass -from datetime import datetime +from datetime import datetime, timedelta from functools import reduce from pathlib import Path from typing import ( @@ -691,6 +691,7 @@ def compact( partition_filters: Optional[FilterType] = None, target_size: Optional[int] = None, max_concurrent_tasks: Optional[int] = None, + min_commit_interval: Optional[Union[int, timedelta]] = None, ) -> Dict[str, Any]: """ Compacts small files to reduce the total number of files in the table. @@ -708,10 +709,25 @@ def compact( :param max_concurrent_tasks: the maximum number of concurrent tasks to use for file compaction. Defaults to number of CPUs. More concurrent tasks can make compaction faster, but will also use more memory. + :param min_commit_interval: minimum interval in seconds or as timedeltas before a new commit is + created. Interval is useful for long running executions. Set to 0 or timedelta(0), if you + want a commit per partition. :return: the metrics from optimize + + Examples: + + Use a timedelta object to specify the seconds, minutes or hours of the interval. + >>> from deltalake import DeltaTable + >>> from datetime import timedelta + >>> dt = DeltaTable("tmp") + >>> time_delta = timedelta(minutes=10) + >>> dt.optimize.z_order(["timestamp"], min_commit_interval=time_delta) """ + if isinstance(min_commit_interval, timedelta): + min_commit_interval = int(min_commit_interval.total_seconds()) + metrics = self.table._table.compact_optimize( - partition_filters, target_size, max_concurrent_tasks + partition_filters, target_size, max_concurrent_tasks, min_commit_interval ) self.table.update_incremental() return json.loads(metrics) @@ -723,6 +739,7 @@ def z_order( target_size: Optional[int] = None, max_concurrent_tasks: Optional[int] = None, max_spill_size: int = 20 * 1024 * 1024 * 1024, + min_commit_interval: Optional[Union[int, timedelta]] = None, ) -> Dict[str, Any]: """ Reorders the data using a Z-order curve to improve data skipping. @@ -738,14 +755,30 @@ def z_order( file compaction. Defaults to number of CPUs. More concurrent tasks can make compaction faster, but will also use more memory. :param max_spill_size: the maximum number of bytes to spill to disk. Defaults to 20GB. + :param min_commit_interval: minimum interval in seconds or as timedeltas before a new commit is + created. Interval is useful for long running executions. Set to 0 or timedelta(0), if you + want a commit per partition. :return: the metrics from optimize + + Examples: + + Use a timedelta object to specify the seconds, minutes or hours of the interval. + >>> from deltalake import DeltaTable + >>> from datetime import timedelta + >>> dt = DeltaTable("tmp") + >>> time_delta = timedelta(minutes=10) + >>> dt.optimize.compact(min_commit_interval=time_delta) """ + if isinstance(min_commit_interval, timedelta): + min_commit_interval = int(min_commit_interval.total_seconds()) + metrics = self.table._table.z_order_optimize( list(columns), partition_filters, target_size, max_concurrent_tasks, max_spill_size, + min_commit_interval, ) self.table.update_incremental() return json.loads(metrics) diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py index 72a6cd929d..c11b808659 100644 --- a/python/docs/source/conf.py +++ b/python/docs/source/conf.py @@ -66,6 +66,7 @@ def get_release_version() -> str: ("py:class", "pyarrow._dataset_parquet.ParquetFileWriteOptions"), ("py:class", "pathlib.Path"), ("py:class", "datetime.datetime"), + ("py:class", "datetime.timedelta"), ] # Add any paths that contain templates here, relative to this directory. diff --git a/python/src/lib.rs b/python/src/lib.rs index b4fc515f2b..8d7cdb486d 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -9,6 +9,7 @@ use std::collections::{HashMap, HashSet}; use std::convert::TryFrom; use std::future::IntoFuture; use std::sync::Arc; +use std::time; use std::time::{SystemTime, UNIX_EPOCH}; use arrow::pyarrow::PyArrowType; @@ -273,18 +274,22 @@ impl RawDeltaTable { } /// Run the optimize command on the Delta Table: merge small files into a large file by bin-packing. - #[pyo3(signature = (partition_filters = None, target_size = None, max_concurrent_tasks = None))] + #[pyo3(signature = (partition_filters = None, target_size = None, max_concurrent_tasks = None, min_commit_interval = None))] pub fn compact_optimize( &mut self, partition_filters: Option>, target_size: Option, max_concurrent_tasks: Option, + min_commit_interval: Option, ) -> PyResult { let mut cmd = OptimizeBuilder::new(self._table.object_store(), self._table.state.clone()) .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)); if let Some(size) = target_size { cmd = cmd.with_target_size(size); } + if let Some(commit_interval) = min_commit_interval { + cmd = cmd.with_min_commit_interval(time::Duration::from_secs(commit_interval)); + } let converted_filters = convert_partition_filters(partition_filters.unwrap_or_default()) .map_err(PythonError::from)?; cmd = cmd.with_filters(&converted_filters); @@ -297,7 +302,7 @@ impl RawDeltaTable { } /// Run z-order variation of optimize - #[pyo3(signature = (z_order_columns, partition_filters = None, target_size = None, max_concurrent_tasks = None, max_spill_size = 20 * 1024 * 1024 * 1024))] + #[pyo3(signature = (z_order_columns, partition_filters = None, target_size = None, max_concurrent_tasks = None, max_spill_size = 20 * 1024 * 1024 * 1024, min_commit_interval = None))] pub fn z_order_optimize( &mut self, z_order_columns: Vec, @@ -305,6 +310,7 @@ impl RawDeltaTable { target_size: Option, max_concurrent_tasks: Option, max_spill_size: usize, + min_commit_interval: Option, ) -> PyResult { let mut cmd = OptimizeBuilder::new(self._table.object_store(), self._table.state.clone()) .with_max_concurrent_tasks(max_concurrent_tasks.unwrap_or_else(num_cpus::get)) @@ -313,6 +319,10 @@ impl RawDeltaTable { if let Some(size) = target_size { cmd = cmd.with_target_size(size); } + if let Some(commit_interval) = min_commit_interval { + cmd = cmd.with_min_commit_interval(time::Duration::from_secs(commit_interval)); + } + let converted_filters = convert_partition_filters(partition_filters.unwrap_or_default()) .map_err(PythonError::from)?; cmd = cmd.with_filters(&converted_filters); diff --git a/python/tests/test_optimize.py b/python/tests/test_optimize.py index 665aaaec8f..4b746b1434 100644 --- a/python/tests/test_optimize.py +++ b/python/tests/test_optimize.py @@ -1,4 +1,5 @@ import pathlib +from datetime import timedelta import pyarrow as pa import pytest @@ -46,3 +47,24 @@ def test_z_order_optimize( last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" assert dt.version() == old_version + 1 + + +def test_optimize_min_commit_interval( + tmp_path: pathlib.Path, + sample_data: pa.Table, +): + write_deltalake(tmp_path, sample_data, partition_by="utf8", mode="append") + write_deltalake(tmp_path, sample_data, partition_by="utf8", mode="append") + write_deltalake(tmp_path, sample_data, partition_by="utf8", mode="append") + + dt = DeltaTable(tmp_path) + old_version = dt.version() + + dt.optimize.z_order(["date32", "timestamp"], min_commit_interval=timedelta(0)) + + last_action = dt.history(1)[0] + assert last_action["operation"] == "OPTIMIZE" + # The table has 5 distinct partitions, each of which are Z-ordered + # independently. So with min_commit_interval=0, each will get its + # own commit. + assert dt.version() == old_version + 5 From 9eccd4c971989187003f8978846725cf44d4cbb2 Mon Sep 17 00:00:00 2001 From: Matthew Powers Date: Sat, 23 Sep 2023 11:03:39 -0500 Subject: [PATCH 08/99] docs: add docstring to protocol method (#1660) --- python/deltalake/table.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 367debbf18..af1cf090da 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -411,6 +411,11 @@ def metadata(self) -> Metadata: return self._metadata def protocol(self) -> ProtocolVersions: + """ + Get the reader and writer protocol versions of the DeltaTable. + + :return: the current ProtocolVersions registered in the transaction log + """ return ProtocolVersions(*self._table.protocol_versions()) def history(self, limit: Optional[int] = None) -> List[Dict[str, Any]]: From 6ef0d7c2bf665f24fac1c6f878a7ec768281b5de Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 21:39:20 +0200 Subject: [PATCH 09/99] fix: percent encoding of partition values and paths --- python/deltalake/writer.py | 25 ++++++++++++++++--- .../test_writer_readable.py | 2 +- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index f2754a760d..8decb41f8e 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -17,6 +17,7 @@ Tuple, Union, ) +from urllib.parse import unquote, quote from deltalake.fs import DeltaStorageHandler @@ -207,12 +208,15 @@ def write_deltalake( partition_schema = pa.schema([schema.field(name) for name in partition_by]) partitioning = ds.partitioning(partition_schema, flavor="hive") else: + partition_schema = pa.schema([]) partitioning = None add_actions: List[AddAction] = [] def visitor(written_file: Any) -> None: - path, partition_values = get_partitions_from_path(written_file.path) + path, partition_values = get_partitions_from_path( + written_file.path, partition_schema=partition_schema + ) stats = get_file_stats_from_metadata(written_file.metadata) # PyArrow added support for written_file.size in 9.0.0 @@ -225,7 +229,7 @@ def visitor(written_file: Any) -> None: add_actions.append( AddAction( - path, + quote(path), size, partition_values, int(datetime.now().timestamp() * 1000), @@ -409,7 +413,17 @@ def try_get_deltatable( return None -def get_partitions_from_path(path: str) -> Tuple[str, Dict[str, Optional[str]]]: +quoted_types = [ + pa.timestamp("s"), + pa.timestamp("ms"), + pa.timestamp("us"), + pa.timestamp("ns"), +] + + +def get_partitions_from_path( + path: str, partition_schema: pa.Schema +) -> Tuple[str, Dict[str, Optional[str]]]: if path[0] == "/": path = path[1:] parts = path.split("/") @@ -422,7 +436,10 @@ def get_partitions_from_path(path: str) -> Tuple[str, Dict[str, Optional[str]]]: if value == "__HIVE_DEFAULT_PARTITION__": out[key] = None else: - out[key] = value + if partition_schema.field(key).type in quoted_types: + out[key] = unquote(value) + else: + out[key] = value return path, out diff --git a/python/tests/pyspark_integration/test_writer_readable.py b/python/tests/pyspark_integration/test_writer_readable.py index f637255951..e9d5603191 100644 --- a/python/tests/pyspark_integration/test_writer_readable.py +++ b/python/tests/pyspark_integration/test_writer_readable.py @@ -34,7 +34,7 @@ def test_basic_read(sample_data: pa.Table, existing_table: DeltaTable): @pytest.mark.pyspark @pytest.mark.integration def test_partitioned(tmp_path: pathlib.Path, sample_data: pa.Table): - partition_cols = ["date32", "utf8"] + partition_cols = ["date32", "utf8", "timestamp", "bool"] # Add null values to sample data to verify we can read null partitions sample_data_with_null = sample_data From 9fef678bc4931eefc671d9bf6c8d386b17b0c8fe Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 09:56:02 +0200 Subject: [PATCH 10/99] feat: handle path encoding in serde and encode partition values in file names --- .gitignore | 1 + rust/Cargo.toml | 1 + rust/examples/basic_operations.rs | 41 +++++++++++++++++++++++------ rust/examples/recordbatch-writer.rs | 2 +- rust/src/action/mod.rs | 34 +++++++++--------------- rust/src/action/serde_path.rs | 19 +++++++++++++ rust/src/operations/restore.rs | 2 +- rust/src/operations/writer.rs | 2 +- rust/src/table_state.rs | 3 +-- rust/src/writer/utils.rs | 14 +++++----- 10 files changed, 77 insertions(+), 42 deletions(-) create mode 100644 rust/src/action/serde_path.rs diff --git a/.gitignore b/.gitignore index 5fe8f6cf0a..8642b9722a 100644 --- a/.gitignore +++ b/.gitignore @@ -27,3 +27,4 @@ Cargo.lock !/delta-inspect/Cargo.lock !/proofs/Cargo.lock +justfile \ No newline at end of file diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 90941f8fe8..77aeb6e940 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -80,6 +80,7 @@ parquet2 = { version = "0.17", optional = true } percent-encoding = "2" tracing = { version = "0.1", optional = true } rand = "0.8" +urlencoding = "2" # hdfs datafusion-objectstore-hdfs = { version = "0.1.3", default-features = false, features = [ diff --git a/rust/examples/basic_operations.rs b/rust/examples/basic_operations.rs index 5c1fb46e86..d95aadfb78 100644 --- a/rust/examples/basic_operations.rs +++ b/rust/examples/basic_operations.rs @@ -1,6 +1,6 @@ use arrow::{ - array::{Int32Array, StringArray}, - datatypes::{DataType, Field, Schema as ArrowSchema}, + array::{Int32Array, StringArray, TimestampMicrosecondArray}, + datatypes::{DataType, Field, Schema as ArrowSchema, TimeUnit}, record_batch::RecordBatch, }; use deltalake::operations::collect_sendable_stream; @@ -26,6 +26,12 @@ fn get_table_columns() -> Vec { true, Default::default(), ), + SchemaField::new( + String::from("timestamp"), + SchemaDataType::primitive(String::from("timestamp")), + true, + Default::default(), + ), ] } @@ -33,20 +39,38 @@ fn get_table_batches() -> RecordBatch { let schema = Arc::new(ArrowSchema::new(vec![ Field::new("int", DataType::Int32, false), Field::new("string", DataType::Utf8, true), + Field::new( + "timestamp", + DataType::Timestamp(TimeUnit::Microsecond, None), + true, + ), ])); let int_values = Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]); let str_values = StringArray::from(vec!["A", "B", "A", "B", "A", "A", "A", "B", "B", "A", "A"]); - - RecordBatch::try_new(schema, vec![Arc::new(int_values), Arc::new(str_values)]).unwrap() + let ts_values = TimestampMicrosecondArray::from(vec![ + 1000000012, 1000000012, 1000000012, 1000000012, 500012305, 500012305, 500012305, 500012305, + 500012305, 500012305, 500012305, + ]); + RecordBatch::try_new( + schema, + vec![ + Arc::new(int_values), + Arc::new(str_values), + Arc::new(ts_values), + ], + ) + .unwrap() } #[tokio::main(flavor = "current_thread")] async fn main() -> Result<(), deltalake::errors::DeltaTableError> { - // Create a delta operations client pointing at an un-initialized in-memory location. - // In a production environment this would be created with "try_new" and point at - // a real storage location. - let ops = DeltaOps::new_in_memory(); + // Create a delta operations client pointing at an un-initialized location. + let ops = if let Ok(table_uri) = std::env::var("TABLE_URI") { + DeltaOps::try_from_uri(table_uri).await? + } else { + DeltaOps::new_in_memory() + }; // The operations module uses a builder pattern that allows specifying several options // on how the command behaves. The builders implement `Into`, so once @@ -54,6 +78,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { let table = ops .create() .with_columns(get_table_columns()) + .with_partition_columns(["timestamp"]) .with_table_name("my_table") .with_comment("A table to show how delta-rs works") .await?; diff --git a/rust/examples/recordbatch-writer.rs b/rust/examples/recordbatch-writer.rs index cca9c6e3fc..f08c33952b 100644 --- a/rust/examples/recordbatch-writer.rs +++ b/rust/examples/recordbatch-writer.rs @@ -36,7 +36,7 @@ async fn main() -> Result<(), DeltaTableError> { })?; info!("Using the location of: {:?}", table_uri); - let table_path = Path::from(table_uri.as_ref()); + let table_path = Path::parse(&table_uri)?; let maybe_table = deltalake::open_table(&table_path).await; let mut table = match maybe_table { diff --git a/rust/src/action/mod.rs b/rust/src/action/mod.rs index d411a502cd..105ebdcdab 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/action/mod.rs @@ -8,6 +8,7 @@ pub mod checkpoints; pub mod parquet2_read; #[cfg(feature = "parquet")] mod parquet_read; +mod serde_path; #[cfg(feature = "arrow")] use arrow_schema::ArrowError; @@ -15,7 +16,6 @@ use futures::StreamExt; use lazy_static::lazy_static; use log::*; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; -use percent_encoding::percent_decode; use regex::Regex; use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; @@ -105,13 +105,6 @@ pub enum ProtocolError { }, } -fn decode_path(raw_path: &str) -> Result { - percent_decode(raw_path.as_bytes()) - .decode_utf8() - .map(|c| c.to_string()) - .map_err(|e| ProtocolError::InvalidField(format!("Decode path failed for action: {e}"))) -} - /// Struct used to represent minValues and maxValues in add action statistics. #[derive(Serialize, Deserialize, Debug, PartialEq, Eq)] #[serde(untagged)] @@ -255,6 +248,7 @@ pub struct StatsParsed { pub struct AddCDCFile { /// A relative path, from the root of the table, or an /// absolute path to a CDC file + #[serde(with = "serde_path")] pub path: String, /// The size of this file in bytes pub size: i64, @@ -351,6 +345,7 @@ impl Eq for DeletionVector {} #[serde(rename_all = "camelCase")] pub struct Add { /// A relative path, from the root of the table, to a file that should be added to the table + #[serde(with = "serde_path")] pub path: String, /// The size of this file in bytes pub size: i64, @@ -403,9 +398,11 @@ pub struct Add { #[serde(skip_serializing, skip_deserializing)] pub stats_parsed: Option, /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, - ///Metadata about deletion vector + /// Metadata about deletion vector + #[serde(skip_serializing_if = "Option::is_none")] pub deletion_vector: Option, } @@ -431,11 +428,6 @@ impl PartialEq for Add { impl Eq for Add {} impl Add { - /// Returns the Add action with path decoded. - pub fn path_decoded(self) -> Result { - decode_path(&self.path).map(|path| Self { path, ..self }) - } - /// Get whatever stats are available. Uses (parquet struct) parsed_stats if present falling back to json stats. #[cfg(any(feature = "parquet", feature = "parquet2"))] pub fn get_stats(&self) -> Result, serde_json::error::Error> { @@ -569,6 +561,7 @@ impl TryFrom for MetaData { #[serde(rename_all = "camelCase")] pub struct Remove { /// The path of the file that is removed from the table. + #[serde(with = "serde_path")] pub path: String, /// The timestamp when the remove was added to table state. pub deletion_timestamp: Option, @@ -581,12 +574,16 @@ pub struct Remove { /// it's still nullable so we keep it as Option<> for compatibly. pub extended_file_metadata: Option, /// A map from partition column to value for this file. + #[serde(skip_serializing_if = "Option::is_none")] pub partition_values: Option>>, /// Size of this file in bytes + #[serde(skip_serializing_if = "Option::is_none")] pub size: Option, /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, - ///Metadata about deletion vector + /// Metadata about deletion vector + #[serde(skip_serializing_if = "Option::is_none")] pub deletion_vector: Option, } @@ -617,13 +614,6 @@ impl PartialEq for Remove { } } -impl Remove { - /// Returns the Remove action with path decoded. - pub fn path_decoded(self) -> Result { - decode_path(&self.path).map(|path| Self { path, ..self }) - } -} - /// Action used by streaming systems to track progress using application-specific versions to /// enable idempotency. #[derive(Serialize, Deserialize, Debug, Default, Clone)] diff --git a/rust/src/action/serde_path.rs b/rust/src/action/serde_path.rs new file mode 100644 index 0000000000..ddc51335e8 --- /dev/null +++ b/rust/src/action/serde_path.rs @@ -0,0 +1,19 @@ +use serde::{self, Deserialize, Deserializer, Serialize, Serializer}; +use urlencoding::{decode, encode}; + +pub fn deserialize<'de, D>(deserializer: D) -> Result +where + D: Deserializer<'de>, +{ + let s = String::deserialize(deserializer)?; + let decoded = decode(&s).map_err(serde::de::Error::custom)?.into_owned(); + Ok(decoded) +} + +pub fn serialize(value: &String, serializer: S) -> Result +where + S: Serializer, +{ + let decoded = encode(value).into_owned(); + String::serialize(&decoded, serializer) +} diff --git a/rust/src/operations/restore.rs b/rust/src/operations/restore.rs index a5e51da05b..5800edd96f 100644 --- a/rust/src/operations/restore.rs +++ b/rust/src/operations/restore.rs @@ -252,7 +252,7 @@ async fn check_files_available( files: &Vec, ) -> DeltaResult<()> { for file in files { - let file_path = Path::from(file.path.clone()); + let file_path = Path::parse(file.path.clone())?; match object_store.head(&file_path).await { Ok(_) => {} Err(ObjectStoreError::NotFound { .. }) => { diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index 4fef892bf8..a72b832505 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -247,7 +247,7 @@ impl PartitionWriterConfig { .map_err(|err| WriteError::FileName { source: Box::new(err), })?; - let prefix = Path::from(part_path.as_ref()); + let prefix = Path::parse(part_path.as_ref())?; let writer_properties = writer_properties.unwrap_or_else(|| { WriterProperties::builder() .set_created_by(format!("delta-rs version {}", crate_version())) diff --git a/rust/src/table_state.rs b/rust/src/table_state.rs index 9be2200d9e..2ac17032d4 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table_state.rs @@ -327,12 +327,11 @@ impl DeltaTableState { action::Action::cdc(_v) => {} action::Action::add(v) => { if require_files { - self.files.push(v.path_decoded()?); + self.files.push(v); } } action::Action::remove(v) => { if require_tombstones && require_files { - let v = v.path_decoded()?; self.tombstones.insert(v); } } diff --git a/rust/src/writer/utils.rs b/rust/src/writer/utils.rs index a12809916f..c7f01846fb 100644 --- a/rust/src/writer/utils.rs +++ b/rust/src/writer/utils.rs @@ -21,6 +21,7 @@ use parquet::basic::Compression; use parquet::file::properties::WriterProperties; use parquet::schema::types::ColumnPath; use serde_json::Value; +use urlencoding::encode; use uuid::Uuid; use crate::errors::DeltaResult; @@ -45,13 +46,12 @@ impl PartitionPath { let partition_value = partition_values .get(k) .ok_or_else(|| DeltaWriterError::MissingPartitionColumn(k.to_string()))?; - - let partition_value = partition_value - .as_deref() - .unwrap_or(NULL_PARTITION_VALUE_DATA_PATH); - let part = format!("{k}={partition_value}"); - - path_parts.push(part); + let partition_value = if let Some(val) = partition_value.as_deref() { + encode(val).into_owned() + } else { + NULL_PARTITION_VALUE_DATA_PATH.to_string() + }; + path_parts.push(format!("{k}={partition_value}")); } Ok(PartitionPath { From 18d7a33627b720826f7e1b6555d1141734f38d2e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 10:44:01 +0200 Subject: [PATCH 11/99] fix: always unquote partition values extracted from path --- python/deltalake/writer.py | 22 +++------------------- 1 file changed, 3 insertions(+), 19 deletions(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 8decb41f8e..63befd7619 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -208,15 +208,12 @@ def write_deltalake( partition_schema = pa.schema([schema.field(name) for name in partition_by]) partitioning = ds.partitioning(partition_schema, flavor="hive") else: - partition_schema = pa.schema([]) partitioning = None add_actions: List[AddAction] = [] def visitor(written_file: Any) -> None: - path, partition_values = get_partitions_from_path( - written_file.path, partition_schema=partition_schema - ) + path, partition_values = get_partitions_from_path(written_file.path) stats = get_file_stats_from_metadata(written_file.metadata) # PyArrow added support for written_file.size in 9.0.0 @@ -413,17 +410,7 @@ def try_get_deltatable( return None -quoted_types = [ - pa.timestamp("s"), - pa.timestamp("ms"), - pa.timestamp("us"), - pa.timestamp("ns"), -] - - -def get_partitions_from_path( - path: str, partition_schema: pa.Schema -) -> Tuple[str, Dict[str, Optional[str]]]: +def get_partitions_from_path(path: str) -> Tuple[str, Dict[str, Optional[str]]]: if path[0] == "/": path = path[1:] parts = path.split("/") @@ -436,10 +423,7 @@ def get_partitions_from_path( if value == "__HIVE_DEFAULT_PARTITION__": out[key] = None else: - if partition_schema.field(key).type in quoted_types: - out[key] = unquote(value) - else: - out[key] = value + out[key] = unquote(value) return path, out From 1aa84291fadb986839aa374126ce18c3247503d3 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 11:10:46 +0200 Subject: [PATCH 12/99] test: add tests for related issues --- python/tests/test_table_read.py | 46 +++++++++++++++++++++++++++++++++ python/tests/test_writer.py | 16 ++++++++++++ 2 files changed, 62 insertions(+) diff --git a/python/tests/test_table_read.py b/python/tests/test_table_read.py index db00d96f26..f56c5876cd 100644 --- a/python/tests/test_table_read.py +++ b/python/tests/test_table_read.py @@ -648,3 +648,49 @@ def assert_scan_equals(table, predicate, expected): assert_num_fragments(table, predicate, 2) expected = pa.table({"part": ["a", "a", "b", "b"], "value": [1, 1, None, None]}) assert_scan_equals(table, predicate, expected) + + +def test_issue_1653_filter_bool_partition(tmp_path: Path): + ta = pa.Table.from_pydict( + { + "bool_col": [True, False, True, False], + "int_col": [0, 1, 2, 3], + "str_col": ["a", "b", "c", "d"], + } + ) + write_deltalake( + tmp_path, ta, partition_by=["bool_col", "int_col"], mode="overwrite" + ) + dt = DeltaTable(tmp_path) + + assert ( + dt.to_pyarrow_table( + filters=[ + ("int_col", "=", 0), + ("bool_col", "=", True), + ] + ).num_rows + == 1 + ) + assert ( + len( + dt.file_uris( + partition_filters=[ + ("int_col", "=", 0), + ("bool_col", "=", "true"), + ] + ) + ) + == 1 + ) + assert ( + len( + dt.file_uris( + partition_filters=[ + ("int_col", "=", 0), + ("bool_col", "=", True), + ] + ) + ) + == 1 + ) diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index e45d56539c..ed3ca98ff3 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -888,3 +888,19 @@ def comp(): "a concurrent transaction deleted the same data your transaction deletes" in str(exception) ) + + +def test_issue_1651_roundtrip_timestamp(tmp_path: pathlib.Path): + data = pa.table( + { + "id": pa.array([425], type=pa.int32()), + "data": pa.array(["python-module-test-write"]), + "t": pa.array([datetime(2023, 9, 15)]), + } + ) + + write_deltalake(table_or_uri=tmp_path, mode="append", data=data, partition_by=["t"]) + dt = DeltaTable(table_uri=tmp_path) + dataset = dt.to_pyarrow_dataset() + + assert dataset.count_rows() == 1 From 2a6492c06e9580705456d66493668d457fa29b9e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 11:16:41 +0200 Subject: [PATCH 13/99] fix: consistent serialization of partition values --- python/deltalake/_util.py | 21 +++++++++++++++++++++ python/deltalake/table.py | 5 +++-- python/deltalake/writer.py | 21 ++------------------- 3 files changed, 26 insertions(+), 21 deletions(-) create mode 100644 python/deltalake/_util.py diff --git a/python/deltalake/_util.py b/python/deltalake/_util.py new file mode 100644 index 0000000000..21fe22d9d7 --- /dev/null +++ b/python/deltalake/_util.py @@ -0,0 +1,21 @@ +from typing import Any + +from datetime import date, datetime + + +def encode_partition_value(val: Any) -> str: + # Rules based on: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#partition-value-serialization + if isinstance(val, bool): + return str(val).lower() + if isinstance(val, str): + return val + elif isinstance(val, (int, float)): + return str(val) + elif isinstance(val, date): + return val.isoformat() + elif isinstance(val, datetime): + return val.isoformat(sep=" ") + elif isinstance(val, bytes): + return val.decode("unicode_escape", "backslashreplace") + else: + raise ValueError(f"Could not encode partition value for type: {val}") diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 367debbf18..4a9faf7920 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -35,6 +35,7 @@ from .exceptions import DeltaProtocolError from .fs import DeltaStorageHandler from .schema import Schema +from ._util import encode_partition_value MAX_SUPPORTED_READER_VERSION = 1 MAX_SUPPORTED_WRITER_VERSION = 2 @@ -620,9 +621,9 @@ def __stringify_partition_values( for field, op, value in partition_filters: str_value: Union[str, List[str]] if isinstance(value, (list, tuple)): - str_value = [str(val) for val in value] + str_value = [encode_partition_value(val) for val in value] else: - str_value = str(value) + str_value = encode_partition_value(value) out.append((field, op, str_value)) return out diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 63befd7619..9f87ec338a 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -20,6 +20,7 @@ from urllib.parse import unquote, quote from deltalake.fs import DeltaStorageHandler +from ._util import encode_partition_value if TYPE_CHECKING: import pandas as pd @@ -263,7 +264,7 @@ def check_data_is_aligned_with_partition_filtering( for i in range(partition_values.num_rows): # Map will maintain order of partition_columns partition_map = { - column_name: __encode_partition_value( + column_name: encode_partition_value( batch.column(column_name)[i].as_py() ) for column_name in table.metadata().partition_columns @@ -490,21 +491,3 @@ def iter_groups(metadata: Any) -> Iterator[Any]: maximum for maximum in maximums if maximum is not None ) return stats - - -def __encode_partition_value(val: Any) -> str: - # Rules based on: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#partition-value-serialization - if isinstance(val, bool): - return str(val).lower() - if isinstance(val, str): - return val - elif isinstance(val, (int, float)): - return str(val) - elif isinstance(val, date): - return val.isoformat() - elif isinstance(val, datetime): - return val.isoformat(sep=" ") - elif isinstance(val, bytes): - return val.decode("unicode_escape", "backslashreplace") - else: - raise ValueError(f"Could not encode partition value for type: {val}") From dda2d9d53b9726a70473cc6e5b8f99788eb43549 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 16:00:42 +0200 Subject: [PATCH 14/99] fix: rounbdtrip special characters --- python/deltalake/writer.py | 2 +- .../test_writer_readable.py | 32 ++++++++ rust/Cargo.toml | 1 - rust/examples/load_table.rs | 20 +++++ rust/src/action/serde_path.rs | 82 +++++++++++++++++-- rust/src/writer/utils.rs | 13 +-- 6 files changed, 136 insertions(+), 14 deletions(-) create mode 100644 rust/examples/load_table.rs diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 9f87ec338a..fc7978213b 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -227,7 +227,7 @@ def visitor(written_file: Any) -> None: add_actions.append( AddAction( - quote(path), + path, size, partition_values, int(datetime.now().timestamp() * 1000), diff --git a/python/tests/pyspark_integration/test_writer_readable.py b/python/tests/pyspark_integration/test_writer_readable.py index e9d5603191..ea555074b8 100644 --- a/python/tests/pyspark_integration/test_writer_readable.py +++ b/python/tests/pyspark_integration/test_writer_readable.py @@ -12,6 +12,7 @@ import delta import delta.pip_utils import delta.tables + import pyspark.pandas as ps spark = get_spark() except ModuleNotFoundError: @@ -63,3 +64,34 @@ def test_overwrite( write_deltalake(path, sample_data, mode="overwrite") assert_spark_read_equal(sample_data, path) + + +@pytest.mark.pyspark +@pytest.mark.integration +def test_issue_1591_roundtrip_special_characters(tmp_path: pathlib.Path): + test_string = r'$%&/()=^"[]#*?.:_-{=}|`<>~/\r\n+' + poisoned = "}|`<>~" + for char in poisoned: + test_string = test_string.replace(char, "") + + data = pa.table( + { + "string": pa.array([test_string], type=pa.utf8()), + "data": pa.array(["python-module-test-write"]), + } + ) + + deltalake_path = tmp_path / "deltalake" + write_deltalake( + table_or_uri=deltalake_path, mode="append", data=data, partition_by=["string"] + ) + + loaded = ps.read_delta(str(deltalake_path), index_col=None).to_pandas() + assert loaded.shape == data.shape + + spark_path = tmp_path / "spark" + spark_df = spark.createDataFrame(data.to_pandas()) + spark_df.write.format("delta").partitionBy(["string"]).save(str(spark_path)) + + loaded = DeltaTable(spark_path).to_pandas() + assert loaded.shape == data.shape diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 77aeb6e940..90941f8fe8 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -80,7 +80,6 @@ parquet2 = { version = "0.17", optional = true } percent-encoding = "2" tracing = { version = "0.1", optional = true } rand = "0.8" -urlencoding = "2" # hdfs datafusion-objectstore-hdfs = { version = "0.1.3", default-features = false, features = [ diff --git a/rust/examples/load_table.rs b/rust/examples/load_table.rs new file mode 100644 index 0000000000..18a960eeb0 --- /dev/null +++ b/rust/examples/load_table.rs @@ -0,0 +1,20 @@ +use arrow::record_batch::RecordBatch; +use deltalake::operations::collect_sendable_stream; +use deltalake::DeltaOps; + +#[tokio::main(flavor = "current_thread")] +async fn main() -> Result<(), deltalake::errors::DeltaTableError> { + // Create a delta operations client pointing at an un-initialized location. + let ops = if let Ok(table_uri) = std::env::var("TABLE_URI") { + DeltaOps::try_from_uri(table_uri).await? + } else { + DeltaOps::try_from_uri("./rust/tests/data/delta-0.8.0").await? + }; + + let (_table, stream) = ops.load().await?; + let data: Vec = collect_sendable_stream(stream).await?; + + println!("{:?}", data); + + Ok(()) +} diff --git a/rust/src/action/serde_path.rs b/rust/src/action/serde_path.rs index ddc51335e8..9868523e81 100644 --- a/rust/src/action/serde_path.rs +++ b/rust/src/action/serde_path.rs @@ -1,19 +1,89 @@ +use std::str::Utf8Error; + +use percent_encoding::{percent_decode_str, percent_encode, AsciiSet, CONTROLS}; use serde::{self, Deserialize, Deserializer, Serialize, Serializer}; -use urlencoding::{decode, encode}; pub fn deserialize<'de, D>(deserializer: D) -> Result where D: Deserializer<'de>, { let s = String::deserialize(deserializer)?; - let decoded = decode(&s).map_err(serde::de::Error::custom)?.into_owned(); - Ok(decoded) + decode_path(&s).map_err(serde::de::Error::custom) } -pub fn serialize(value: &String, serializer: S) -> Result +pub fn serialize(value: &str, serializer: S) -> Result where S: Serializer, { - let decoded = encode(value).into_owned(); - String::serialize(&decoded, serializer) + let encoded = encode_path(value); + String::serialize(&encoded, serializer) +} + +pub const _DELIMITER: &str = "/"; +/// The path delimiter as a single byte +pub const _DELIMITER_BYTE: u8 = _DELIMITER.as_bytes()[0]; + +/// Characters we want to encode. +const INVALID: &AsciiSet = &CONTROLS + // The delimiter we are reserving for internal hierarchy + // .add(DELIMITER_BYTE) + // Characters AWS recommends avoiding for object keys + // https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingMetadata.html + .add(b'\\') + .add(b'{') + .add(b'^') + .add(b'}') + .add(b'%') + .add(b'`') + .add(b']') + .add(b'"') + .add(b'>') + .add(b'[') + // .add(b'~') + .add(b'<') + .add(b'#') + .add(b'|') + // Characters Google Cloud Storage recommends avoiding for object names + // https://cloud.google.com/storage/docs/naming-objects + .add(b'\r') + .add(b'\n') + .add(b'*') + .add(b'?'); + +fn encode_path(path: &str) -> String { + percent_encode(path.as_bytes(), INVALID).to_string() +} + +fn decode_path(path: &str) -> Result { + Ok(percent_decode_str(path).decode_utf8()?.to_string()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_encode_path() { + let cases = [ + ( + "string=$%25&%2F()%3D%5E%22%5B%5D%23%2A%3F.%3A/part-00023-4b06bc90-0678-4a63-94a2-f09af1adb945.c000.snappy.parquet", + "string=$%2525&%252F()%253D%255E%2522%255B%255D%2523%252A%253F.%253A/part-00023-4b06bc90-0678-4a63-94a2-f09af1adb945.c000.snappy.parquet", + ), + ( + "string=$%25&%2F()%3D%5E%22<>~%5B%5D%7B}`%23|%2A%3F%2F%5Cr%5Cn.%3A/part-00023-e0a68495-8098-40a6-be5f-b502b111b789.c000.snappy.parquet", + "string=$%2525&%252F()%253D%255E%2522%3C%3E~%255B%255D%257B%7D%60%2523%7C%252A%253F%252F%255Cr%255Cn.%253A/part-00023-e0a68495-8098-40a6-be5f-b502b111b789.c000.snappy.parquet" + ), + ( + "string=$%25&%2F()%3D%5E%22<>~%5B%5D%7B}`%23|%2A%3F%2F%5Cr%5Cn.%3A_-/part-00023-346b6795-dafa-4948-bda5-ecdf4baa4445.c000.snappy.parquet", + "string=$%2525&%252F()%253D%255E%2522%3C%3E~%255B%255D%257B%7D%60%2523%7C%252A%253F%252F%255Cr%255Cn.%253A_-/part-00023-346b6795-dafa-4948-bda5-ecdf4baa4445.c000.snappy.parquet" + ) + ]; + + for (raw, expected) in cases { + let encoded = encode_path(raw); + assert_eq!(encoded, expected); + let decoded = decode_path(expected).unwrap(); + assert_eq!(decoded, raw); + } + } } diff --git a/rust/src/writer/utils.rs b/rust/src/writer/utils.rs index c7f01846fb..ca86ca3a64 100644 --- a/rust/src/writer/utils.rs +++ b/rust/src/writer/utils.rs @@ -15,13 +15,12 @@ use arrow::datatypes::{ }; use arrow::json::ReaderBuilder; use arrow::record_batch::*; -use object_store::path::Path; +use object_store::path::{Path, PathPart}; use parking_lot::RwLock; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; use parquet::schema::types::ColumnPath; use serde_json::Value; -use urlencoding::encode; use uuid::Uuid; use crate::errors::DeltaResult; @@ -46,12 +45,14 @@ impl PartitionPath { let partition_value = partition_values .get(k) .ok_or_else(|| DeltaWriterError::MissingPartitionColumn(k.to_string()))?; - let partition_value = if let Some(val) = partition_value.as_deref() { - encode(val).into_owned() + let path_part = if let Some(val) = partition_value.as_deref() { + let part = PathPart::from(val); + let encoded = part.as_ref(); + format!("{k}={encoded}") } else { - NULL_PARTITION_VALUE_DATA_PATH.to_string() + format!("{k}={NULL_PARTITION_VALUE_DATA_PATH}") }; - path_parts.push(format!("{k}={partition_value}")); + path_parts.push(path_part); } Ok(PartitionPath { From 839d1d73b6c5ae9bfc8af970750cdef02674fde9 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 16:05:16 +0200 Subject: [PATCH 15/99] chore: format --- python/deltalake/_util.py | 3 +-- python/deltalake/table.py | 2 +- python/deltalake/writer.py | 3 ++- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/deltalake/_util.py b/python/deltalake/_util.py index 21fe22d9d7..32b8a20a1b 100644 --- a/python/deltalake/_util.py +++ b/python/deltalake/_util.py @@ -1,6 +1,5 @@ -from typing import Any - from datetime import date, datetime +from typing import Any def encode_partition_value(val: Any) -> str: diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 4a9faf7920..9214c60994 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -31,11 +31,11 @@ import pandas from ._internal import RawDeltaTable +from ._util import encode_partition_value from .data_catalog import DataCatalog from .exceptions import DeltaProtocolError from .fs import DeltaStorageHandler from .schema import Schema -from ._util import encode_partition_value MAX_SUPPORTED_READER_VERSION = 1 MAX_SUPPORTED_WRITER_VERSION = 2 diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index fc7978213b..1cce4d3f68 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -17,9 +17,10 @@ Tuple, Union, ) -from urllib.parse import unquote, quote +from urllib.parse import unquote from deltalake.fs import DeltaStorageHandler + from ._util import encode_partition_value if TYPE_CHECKING: From 4ed5ac0fec098c7d8c6490247923f7f7a5001592 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 16:08:41 +0200 Subject: [PATCH 16/99] fix: add feature requirement to load example --- rust/Cargo.toml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 90941f8fe8..9a20c8e864 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -185,6 +185,10 @@ harness = false name = "basic_operations" required-features = ["datafusion"] +[[example]] +name = "load_table" +required-features = ["datafusion"] + [[example]] name = "recordbatch-writer" required-features = ["arrow"] From 815b075ef629762ea2ac3a3c81b49382583ad0db Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 17:56:08 +0200 Subject: [PATCH 17/99] test: add timestamp col to partitioned roundtrip tests --- python/tests/test_writer.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index ed3ca98ff3..e72d0ac8cd 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -168,6 +168,7 @@ def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): "bool", "binary", "date32", + "timestamp", ], ) def test_roundtrip_partitioned( From c02c4c838ca198336b2b5a32786b131130164442 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 18:50:54 +0200 Subject: [PATCH 18/99] test: add rust roundtip test for special characters --- rust/src/operations/write.rs | 45 ++++++++++++++++++++++++++++++++++-- 1 file changed, 43 insertions(+), 2 deletions(-) diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index 41dfeaae5a..a933700cac 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -552,7 +552,8 @@ fn cast_record_batch( #[cfg(test)] mod tests { use super::*; - use crate::operations::DeltaOps; + use crate::action::SaveMode; + use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::{ get_delta_schema, get_delta_schema_with_nested_struct, get_record_batch, @@ -562,7 +563,7 @@ mod tests { use arrow::datatypes::Schema as ArrowSchema; use arrow_array::{Int32Array, StringArray, TimestampMicrosecondArray}; use arrow_schema::{DataType, TimeUnit}; - use datafusion::assert_batches_sorted_eq; + use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; use serde_json::{json, Value}; #[tokio::test] @@ -855,4 +856,44 @@ mod tests { &expected ); } + + #[tokio::test] + async fn test_special_characters_write_read() { + let tmp_dir = tempdir::TempDir::new("test").unwrap(); + let tmp_path = std::fs::canonicalize(tmp_dir.path()).unwrap(); + + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("string", DataType::Utf8, true), + Field::new("data", DataType::Utf8, true), + ])); + + let str_values = StringArray::from(vec![r#"$%&/()=^"[]#*?.:_- {=}|`<>~/\r\n+"#]); + let data_values = StringArray::from(vec!["test"]); + + let batch = RecordBatch::try_new(schema, vec![Arc::new(str_values), Arc::new(data_values)]) + .unwrap(); + + let ops = DeltaOps::try_from_uri(tmp_path.as_os_str().to_str().unwrap()) + .await + .unwrap(); + + let table = ops + .write([batch.clone()]) + .with_partition_columns(["string"]) + .await + .unwrap(); + + let (_table, stream) = DeltaOps(table).load().await.unwrap(); + let data: Vec = collect_sendable_stream(stream).await.unwrap(); + + let expected = vec![ + "+------+-----------------------------------+", + "| data | string |", + "+------+-----------------------------------+", + r#"| test | $%&/()=^"[]#*?.:_- {=}|`<>~/\r\n+ |"#, + "+------+-----------------------------------+", + ]; + + assert_batches_eq!(&expected, &data); + } } From f9842cdc30691be2772917efaaa79b7df6d2a473 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 24 Sep 2023 19:08:17 +0200 Subject: [PATCH 19/99] fix: encode characters illegal on windows --- rust/src/operations/write.rs | 5 ++++- rust/src/writer/utils.rs | 31 ++++++++++++++++++++++++++++--- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index a933700cac..ca96134935 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -877,12 +877,15 @@ mod tests { .await .unwrap(); - let table = ops + let _table = ops .write([batch.clone()]) .with_partition_columns(["string"]) .await .unwrap(); + let table = crate::open_table(tmp_path.as_os_str().to_str().unwrap()) + .await + .unwrap(); let (_table, stream) = DeltaOps(table).load().await.unwrap(); let data: Vec = collect_sendable_stream(stream).await.unwrap(); diff --git a/rust/src/writer/utils.rs b/rust/src/writer/utils.rs index ca86ca3a64..cfc089c164 100644 --- a/rust/src/writer/utils.rs +++ b/rust/src/writer/utils.rs @@ -15,11 +15,13 @@ use arrow::datatypes::{ }; use arrow::json::ReaderBuilder; use arrow::record_batch::*; -use object_store::path::{Path, PathPart}; +use object_store::path::Path; +use object_store::path::DELIMITER_BYTE; use parking_lot::RwLock; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; use parquet::schema::types::ColumnPath; +use percent_encoding::{percent_encode, AsciiSet, CONTROLS}; use serde_json::Value; use uuid::Uuid; @@ -46,8 +48,7 @@ impl PartitionPath { .get(k) .ok_or_else(|| DeltaWriterError::MissingPartitionColumn(k.to_string()))?; let path_part = if let Some(val) = partition_value.as_deref() { - let part = PathPart::from(val); - let encoded = part.as_ref(); + let encoded = percent_encode(val.as_bytes(), INVALID).to_string(); format!("{k}={encoded}") } else { format!("{k}={NULL_PARTITION_VALUE_DATA_PATH}") @@ -61,6 +62,30 @@ impl PartitionPath { } } +const INVALID: &AsciiSet = &CONTROLS + // everything object store needs encoded ... + .add(DELIMITER_BYTE) + .add(b'\\') + .add(b'{') + .add(b'^') + .add(b'}') + .add(b'%') + .add(b'`') + .add(b']') + .add(b'"') + .add(b'>') + .add(b'[') + .add(b'~') + .add(b'<') + .add(b'#') + .add(b'|') + .add(b'\r') + .add(b'\n') + .add(b'*') + .add(b'?') + //... and some more chars illegal on windows + .add(b':'); + impl From for String { fn from(path: PartitionPath) -> String { path.path From 62e853a9bf728f58596fd38d41663766c0686230 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris Date: Sun, 24 Sep 2023 22:11:32 +0200 Subject: [PATCH 20/99] docs: fix some typos (#1662) # Description Saw two typos and marking merge in rust as half-done with a comment on it's current limitation. --- README.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index d75681de12..fdb5eb4a89 100644 --- a/README.md +++ b/README.md @@ -139,7 +139,7 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc | S3 - R2 | ![done] | ![done] | requires lock for concurrent writes | | Azure Blob | ![done] | ![done] | | | Azure ADLS Gen2 | ![done] | ![done] | | -| Micorosft OneLake | [![open]][onelake-rs] | [![open]][onelake-rs] | | +| Microsoft OneLake | [![open]][onelake-rs] | [![open]][onelake-rs] | | | Google Cloud Storage | ![done] | ![done] | | ### Supported Operations @@ -153,7 +153,7 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc | Delete - predicates | ![done] | | Delete data based on a predicate | | Optimize - compaction | ![done] | ![done] | Harmonize the size of data file | | Optimize - Z-order | ![done] | ![done] | Place similar data into the same file | -| Merge | [![open]][merge-rs] | [![open]][merge-py] | | +| Merge | [![semi-done]][merge-rs]| [![open]][merge-py] | Merge two tables (limited to full re-write) | | FS check | ![done] | | Remove corrupted files from table | ### Protocol Support Level @@ -173,13 +173,14 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc | Reader Version | Requirement | Status | | -------------- | ----------------------------------- | ------ | -| Version 2 | Collumn Mapping | | +| Version 2 | Column Mapping | | | Version 3 | Table Features (requires reader V7) | | [datafusion]: https://github.com/apache/arrow-datafusion [ballista]: https://github.com/apache/arrow-ballista [polars]: https://github.com/pola-rs/polars [open]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/IssueNeutral.svg +[semi-done]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/ApprovedChangesGrey.svg [done]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/ApprovedChanges.svg [roadmap]: https://github.com/delta-io/delta-rs/issues/1128 [merge-py]: https://github.com/delta-io/delta-rs/issues/1357 From 02b3cea58505212454509828660dd534eb2ba6e6 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 16 Aug 2023 21:23:39 +0200 Subject: [PATCH 21/99] feat: use url parsing from object store --- rust/Cargo.toml | 2 +- rust/src/data_catalog/storage/mod.rs | 4 +- rust/src/storage/config.rs | 326 ++++++++++++++------------- rust/src/storage/mod.rs | 4 +- 4 files changed, 175 insertions(+), 161 deletions(-) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 90941f8fe8..c801e11526 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -108,7 +108,7 @@ dashmap = { version = "5", optional = true } sqlparser = { version = "0.37", optional = true } # NOTE dependencies only for integration tests -fs_extra = { version = "1.2.0", optional = true } +fs_extra = { version = "1.3.0", optional = true } tempdir = { version = "0", optional = true } dynamodb_lock = { version = "0", default-features = false, optional = true } diff --git a/rust/src/data_catalog/storage/mod.rs b/rust/src/data_catalog/storage/mod.rs index 726afee102..90d9db8fdb 100644 --- a/rust/src/data_catalog/storage/mod.rs +++ b/rust/src/data_catalog/storage/mod.rs @@ -46,9 +46,9 @@ impl ListingSchemaProvider { storage_options: Option>, ) -> DeltaResult { let uri = ensure_table_uri(root_uri)?; - let storage_options = storage_options.unwrap_or_default().into(); + let mut storage_options = storage_options.unwrap_or_default().into(); // We already parsed the url, so unwrapping is safe. - let store = configure_store(&uri, &storage_options)?; + let store = configure_store(&uri, &mut storage_options)?; Ok(Self { authority: uri.to_string(), store, diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index a9859b2c9b..91525e1cdd 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -2,14 +2,13 @@ use std::collections::HashMap; use std::sync::Arc; -use object_store::memory::InMemory; +use object_store::local::LocalFileSystem; use object_store::path::Path; use object_store::prefix::PrefixStore; -use object_store::{DynObjectStore, ObjectStore}; +use object_store::{parse_url_opts, DynObjectStore, Error as ObjectStoreError}; use serde::{Deserialize, Serialize}; use url::Url; -use super::file::FileStorageBackend; use super::utils::str_is_truthy; use crate::errors::{DeltaResult, DeltaTableError}; @@ -18,11 +17,11 @@ use super::s3::{S3StorageBackend, S3StorageOptions}; #[cfg(feature = "hdfs")] use datafusion_objectstore_hdfs::object_store::hdfs::HadoopFileSystem; #[cfg(any(feature = "s3", feature = "s3-native-tls"))] -use object_store::aws::{AmazonS3Builder, AmazonS3ConfigKey}; +use object_store::aws::AmazonS3ConfigKey; #[cfg(feature = "azure")] -use object_store::azure::{AzureConfigKey, MicrosoftAzureBuilder}; +use object_store::azure::AzureConfigKey; #[cfg(feature = "gcs")] -use object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; +use object_store::gcp::GoogleConfigKey; #[cfg(any( feature = "s3", feature = "s3-native-tls", @@ -31,6 +30,65 @@ use object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; ))] use std::str::FromStr; +/// Recognises various URL formats, identifying the relevant [`ObjectStore`](crate::ObjectStore) +#[derive(Debug, Eq, PartialEq)] +enum ObjectStoreScheme { + /// Url corresponding to LocalFileSystem + Local, + /// Url corresponding to InMemory + Memory, + /// Url corresponding to S3 + AmazonS3, + /// Url corresponding to GoogleCloudStorage + GoogleCloudStorage, + /// Url corresponding to MicrosoftAzure + MicrosoftAzure, + /// Url corresponding to HttpStore + Http, + /// Url corresponding to Hdfs + Hdfs, +} + +impl ObjectStoreScheme { + /// Create an [`ObjectStoreScheme`] from the provided [`Url`] + /// + /// Returns the [`ObjectStoreScheme`] and the remaining [`Path`] + fn parse(url: &Url) -> Result<(Self, Path), ObjectStoreError> { + let strip_bucket = || Some(url.path().strip_prefix('/')?.split_once('/')?.1); + + let (scheme, path) = match (url.scheme(), url.host_str()) { + ("file", None) => (Self::Local, url.path()), + ("memory", None) => (Self::Memory, url.path()), + ("s3" | "s3a", Some(_)) => (Self::AmazonS3, url.path()), + ("gs", Some(_)) => (Self::GoogleCloudStorage, url.path()), + ("az" | "adl" | "azure" | "abfs" | "abfss", Some(_)) => { + (Self::MicrosoftAzure, url.path()) + } + ("http", Some(_)) => (Self::Http, url.path()), + ("hdfs", Some(_)) => (Self::Hdfs, url.path()), + ("https", Some(host)) => { + if host.ends_with("dfs.core.windows.net") || host.ends_with("blob.core.windows.net") + { + (Self::MicrosoftAzure, url.path()) + } else if host.ends_with("amazonaws.com") { + match host.starts_with("s3") { + true => (Self::AmazonS3, strip_bucket().unwrap_or_default()), + false => (Self::AmazonS3, url.path()), + } + } else if host.ends_with("r2.cloudflarestorage.com") { + (Self::AmazonS3, strip_bucket().unwrap_or_default()) + } else { + (Self::Http, url.path()) + } + } + _ => return Err(ObjectStoreError::NotImplemented), + }; + + let path = Path::parse(path)?; + Ok((scheme, path)) + } +} + /// Options used for configuring backend storage #[derive(Clone, Debug, Serialize, Deserialize, Default)] pub struct StorageOptions(pub HashMap); @@ -51,6 +109,45 @@ impl StorageOptions { Self(options) } + /// Add values from the environment to storage options + #[cfg(feature = "azure")] + pub fn with_env_azure(&mut self) { + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if let Ok(config_key) = AzureConfigKey::from_str(&key.to_ascii_lowercase()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + + /// Add values from the environment to storage options + #[cfg(feature = "gcs")] + pub fn with_env_gcs(&mut self) { + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if let Ok(config_key) = GoogleConfigKey::from_str(&key.to_ascii_lowercase()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + + /// Add values from the environment to storage options + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + pub fn with_env_s3(&mut self) { + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } + } + } + } + /// Denotes if unsecure connections via http are allowed pub fn allow_http(&self) -> bool { self.0.iter().any(|(key, value)| { @@ -102,164 +199,81 @@ impl From> for StorageOptions { } pub(crate) fn configure_store( url: &Url, - options: &StorageOptions, + options: &mut StorageOptions, ) -> DeltaResult> { - match url.scheme() { - "file" => try_configure_local( + let (scheme, _prefix) = ObjectStoreScheme::parse(url)?; + match scheme { + ObjectStoreScheme::Local => Ok(Arc::new(LocalFileSystem::new_with_prefix( url.to_file_path() .map_err(|_| DeltaTableError::InvalidTableLocation(url.to_string()))? .to_str() .ok_or_else(|| DeltaTableError::InvalidTableLocation(url.to_string()))?, - ), - "memory" => try_configure_memory(url), - "az" | "abfs" | "abfss" | "azure" | "wasb" | "wasbs" | "adl" => { - try_configure_azure(url, options) + )?)), + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + ObjectStoreScheme::AmazonS3 => { + if options.0.is_empty() { + options.with_env_s3(); + } + if let Ok((store, prefix)) = parse_url_opts(url, options.as_s3_options()) { + let store = S3StorageBackend::try_new( + Arc::new(store), + S3StorageOptions::from_map(&options.0), + )?; + Ok(Arc::new(PrefixStore::new(store, prefix))) + } else { + options.with_env_s3(); + let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; + let store = S3StorageBackend::try_new( + Arc::new(store), + S3StorageOptions::from_map(&options.0), + )?; + Ok(Arc::new(PrefixStore::new(store, prefix))) + } } - "s3" | "s3a" => try_configure_s3(url, options), - "gs" => try_configure_gcs(url, options), - "hdfs" => try_configure_hdfs(url, options), - "https" => { - let host = url.host_str().unwrap_or_default(); - if host.contains("amazonaws.com") { - try_configure_s3(url, options) - } else if host.contains("dfs.core.windows.net") - || host.contains("blob.core.windows.net") - || host.contains("dfs.fabric.microsoft.com") - || host.contains("blob.fabric.microsoft.com") - { - try_configure_azure(url, options) + #[cfg(feature = "azure")] + ObjectStoreScheme::MicrosoftAzure => { + if options.0.is_empty() { + options.with_env_azure(); + } + if let Ok((store, prefix)) = parse_url_opts(url, options.as_azure_options()) { + Ok(Arc::new(PrefixStore::new(store, prefix))) } else { - Err(DeltaTableError::Generic(format!( - "unsupported url: {}", - url.as_str() - ))) + options.with_env_azure(); + let (store, prefix) = parse_url_opts(url, options.as_azure_options())?; + Ok(Arc::new(PrefixStore::new(store, prefix))) } } - _ => Err(DeltaTableError::Generic(format!( - "unsupported url: {}", - url.as_str() - ))), - } -} - -fn try_configure_local>(path: P) -> DeltaResult> { - Ok(Arc::new(FileStorageBackend::try_new(path.as_ref())?)) -} - -fn try_configure_memory(storage_url: &Url) -> DeltaResult> { - url_prefix_handler(InMemory::new(), storage_url) -} - -#[cfg(feature = "gcs")] -#[allow(deprecated)] -fn try_configure_gcs( - storage_url: &Url, - options: &StorageOptions, -) -> DeltaResult> { - let store = GoogleCloudStorageBuilder::from_env() - .with_url(storage_url.as_ref()) - .try_with_options(&options.as_gcs_options())? - .build()?; - url_prefix_handler(store, storage_url) -} - -#[cfg(not(feature = "gcs"))] -fn try_configure_gcs( - storage_url: &Url, - _options: &StorageOptions, -) -> DeltaResult> { - Err(DeltaTableError::MissingFeature { - feature: "gcs", - url: storage_url.as_ref().into(), - }) -} - -#[cfg(feature = "azure")] -#[allow(deprecated)] -fn try_configure_azure( - storage_url: &Url, - options: &StorageOptions, -) -> DeltaResult> { - let store = MicrosoftAzureBuilder::from_env() - .with_url(storage_url.as_ref()) - .try_with_options(&options.as_azure_options())? - .with_allow_http(options.allow_http()) - .build()?; - url_prefix_handler(store, storage_url) -} - -#[cfg(not(feature = "azure"))] -fn try_configure_azure( - storage_url: &Url, - _options: &StorageOptions, -) -> DeltaResult> { - Err(DeltaTableError::MissingFeature { - feature: "azure", - url: storage_url.as_ref().into(), - }) -} - -#[cfg(any(feature = "s3", feature = "s3-native-tls"))] -#[allow(deprecated)] -fn try_configure_s3( - storage_url: &Url, - options: &StorageOptions, -) -> DeltaResult> { - let amazon_s3 = AmazonS3Builder::from_env() - .with_url(storage_url.as_ref()) - .try_with_options(&options.as_s3_options())? - .with_allow_http(options.allow_http()) - .build()?; - let store = - S3StorageBackend::try_new(Arc::new(amazon_s3), S3StorageOptions::from_map(&options.0))?; - url_prefix_handler(store, storage_url) -} - -#[cfg(not(any(feature = "s3", feature = "s3-native-tls")))] -fn try_configure_s3( - storage_url: &Url, - _options: &StorageOptions, -) -> DeltaResult> { - Err(DeltaTableError::MissingFeature { - feature: "s3", - url: storage_url.as_ref().into(), - }) -} - -#[cfg(feature = "hdfs")] -fn try_configure_hdfs( - storage_url: &Url, - _options: &StorageOptions, -) -> DeltaResult> { - let store = HadoopFileSystem::new(storage_url.as_ref()).ok_or_else(|| { - DeltaTableError::Generic(format!( - "failed to create HadoopFileSystem for {}", - storage_url.as_ref() - )) - })?; - url_prefix_handler(store, storage_url) -} - -#[cfg(not(feature = "hdfs"))] -fn try_configure_hdfs( - storage_url: &Url, - _options: &StorageOptions, -) -> DeltaResult> { - Err(DeltaTableError::MissingFeature { - feature: "hdfs", - url: storage_url.as_ref().into(), - }) -} - -fn url_prefix_handler( - store: T, - storage_url: &Url, -) -> DeltaResult> { - let prefix = Path::parse(storage_url.path())?; - if prefix != Path::from("/") { - Ok(Arc::new(PrefixStore::new(store, prefix))) - } else { - Ok(Arc::new(store)) + #[cfg(feature = "gcs")] + ObjectStoreScheme::GoogleCloudStorage => { + if options.0.is_empty() { + options.with_env_gcs(); + } + if let Ok((store, prefix)) = parse_url_opts(url, options.as_gcs_options()) { + Ok(Arc::new(PrefixStore::new(store, prefix))) + } else { + let (store, prefix) = parse_url_opts(url, options.as_gcs_options())?; + Ok(Arc::new(PrefixStore::new(store, prefix))) + } + } + #[cfg(feature = "hdfs")] + ObjectStoreScheme::Hdfs => { + let store = HadoopFileSystem::new(url.as_ref()).ok_or_else(|| { + DeltaTableError::Generic(format!( + "failed to create HadoopFileSystem for {}", + url.as_ref() + )) + })?; + Ok(Arc::new(PrefixStore::new(store, _prefix))) + } + #[cfg(not(feature = "hdfs"))] + ObjectStoreScheme::Hdfs => Err(DeltaTableError::MissingFeature { + feature: "hdfs", + url: url.as_ref().into(), + }), + _ => { + let (store, prefix) = parse_url_opts(url, options.0.clone())?; + Ok(Arc::new(PrefixStore::new(store, prefix))) + } } } @@ -277,7 +291,7 @@ mod test { let table_uri = ensure_table_uri(path.as_os_str().to_str().unwrap()).unwrap(); - let store = configure_store(&table_uri, &StorageOptions::default()).unwrap(); + let store = configure_store(&table_uri, &mut StorageOptions::default()).unwrap(); let contents = b"test"; let key = "test.txt"; diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index 9798a10b1a..8589c3c179 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -91,8 +91,8 @@ impl DeltaObjectStore { /// * `location` - A url pointing to the root of the delta table. /// * `options` - Options passed to underlying builders. See [`with_storage_options`](crate::builder::DeltaTableBuilder::with_storage_options) pub fn try_new(location: Url, options: impl Into + Clone) -> DeltaResult { - let options = options.into(); - let storage = config::configure_store(&location, &options)?; + let mut options = options.into(); + let storage = config::configure_store(&location, &mut options)?; Ok(Self { storage, location, From c7f2e1ae639effb933ce57605371a3d62d805789 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 22 Sep 2023 23:09:08 +0200 Subject: [PATCH 22/99] fix: ensure config for ms fabric --- rust/src/storage/config.rs | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index 91525e1cdd..db6eac6039 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -53,7 +53,7 @@ impl ObjectStoreScheme { /// Create an [`ObjectStoreScheme`] from the provided [`Url`] /// /// Returns the [`ObjectStoreScheme`] and the remaining [`Path`] - fn parse(url: &Url) -> Result<(Self, Path), ObjectStoreError> { + fn parse(url: &Url, options: &mut StorageOptions) -> Result<(Self, Path), ObjectStoreError> { let strip_bucket = || Some(url.path().strip_prefix('/')?.split_once('/')?.1); let (scheme, path) = match (url.scheme(), url.host_str()) { @@ -70,6 +70,19 @@ impl ObjectStoreScheme { if host.ends_with("dfs.core.windows.net") || host.ends_with("blob.core.windows.net") { (Self::MicrosoftAzure, url.path()) + } else if host.contains("dfs.fabric.microsoft.com") + || host.contains("blob.fabric.microsoft.com") + { + if !options + .as_azure_options() + .contains_key(&AzureConfigKey::UseFabricEndpoint) + { + options.0.insert( + AzureConfigKey::UseFabricEndpoint.as_ref().to_string(), + "true".to_string(), + ); + } + (Self::MicrosoftAzure, url.path()) } else if host.ends_with("amazonaws.com") { match host.starts_with("s3") { true => (Self::AmazonS3, strip_bucket().unwrap_or_default()), @@ -197,11 +210,12 @@ impl From> for StorageOptions { Self::new(value) } } + pub(crate) fn configure_store( url: &Url, options: &mut StorageOptions, ) -> DeltaResult> { - let (scheme, _prefix) = ObjectStoreScheme::parse(url)?; + let (scheme, _prefix) = ObjectStoreScheme::parse(url, options)?; match scheme { ObjectStoreScheme::Local => Ok(Arc::new(LocalFileSystem::new_with_prefix( url.to_file_path() From 9db777b6720aa6986a317d01f78e43145eaed671 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 22 Sep 2023 23:20:57 +0200 Subject: [PATCH 23/99] chore: drive-by simplify test files --- rust/tests/command_vacuum.rs | 29 ++++++++++++++++++++++++++++- rust/tests/common/mod.rs | 1 - rust/tests/common/schemas.rs | 28 ---------------------------- 3 files changed, 28 insertions(+), 30 deletions(-) delete mode 100644 rust/tests/common/schemas.rs diff --git a/rust/tests/command_vacuum.rs b/rust/tests/command_vacuum.rs index ce6c6071c7..4437e9dc85 100644 --- a/rust/tests/command_vacuum.rs +++ b/rust/tests/command_vacuum.rs @@ -1,14 +1,41 @@ use chrono::Duration; use common::clock::TestClock; -use common::schemas::{get_vacuum_underscore_schema, get_xy_date_schema}; use common::TestContext; use deltalake::operations::vacuum::Clock; use deltalake::operations::DeltaOps; +use deltalake::Schema; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; +use serde_json::json; use std::sync::Arc; mod common; +/// Basic schema +pub fn get_xy_date_schema() -> Schema { + serde_json::from_value(json!({ + "type": "struct", + "fields": [ + {"name": "x", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "y", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "date", "type": "string", "nullable": false, "metadata": {}}, + ] + })) + .unwrap() +} + +/// Schema that contains a column prefiexed with _ +pub fn get_vacuum_underscore_schema() -> Schema { + serde_json::from_value::(json!({ + "type": "struct", + "fields": [ + {"name": "x", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "y", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "_date", "type": "string", "nullable": false, "metadata": {}}, + ] + })) + .unwrap() +} + #[tokio::test] // Validate vacuum works on a non-partitioned table async fn test_non_partitioned_table() { diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index 2966b1a911..b54fae20d0 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -22,7 +22,6 @@ pub mod datafusion; pub mod hdfs; #[cfg(any(feature = "s3", feature = "s3-native-tls"))] pub mod s3; -pub mod schemas; #[derive(Default)] pub struct TestContext { diff --git a/rust/tests/common/schemas.rs b/rust/tests/common/schemas.rs deleted file mode 100644 index 488c13834f..0000000000 --- a/rust/tests/common/schemas.rs +++ /dev/null @@ -1,28 +0,0 @@ -use deltalake::Schema; -use serde_json::json; - -/// Basic schema -pub fn get_xy_date_schema() -> Schema { - serde_json::from_value(json!({ - "type": "struct", - "fields": [ - {"name": "x", "type": "integer", "nullable": false, "metadata": {}}, - {"name": "y", "type": "integer", "nullable": false, "metadata": {}}, - {"name": "date", "type": "string", "nullable": false, "metadata": {}}, - ] - })) - .unwrap() -} - -/// Schema that contains a column prefiexed with _ -pub fn get_vacuum_underscore_schema() -> Schema { - serde_json::from_value::(json!({ - "type": "struct", - "fields": [ - {"name": "x", "type": "integer", "nullable": false, "metadata": {}}, - {"name": "y", "type": "integer", "nullable": false, "metadata": {}}, - {"name": "_date", "type": "string", "nullable": false, "metadata": {}}, - ] - })) - .unwrap() -} From 84cb6b7f9e369566f6cb2bc0027a00dd29b80585 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 00:05:41 +0200 Subject: [PATCH 24/99] fix: update aws http config key --- .github/workflows/build.yml | 2 +- python/tests/conftest.py | 2 +- python/tests/test_fs.py | 2 +- rust/src/builder.rs | 2 +- rust/src/storage/config.rs | 2 +- rust/tests/common/s3.rs | 4 ++-- rust/tests/integration_datafusion.rs | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index ba2915cdc8..10f5e95691 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -102,7 +102,7 @@ jobs: AWS_ACCESS_KEY_ID: deltalake AWS_SECRET_ACCESS_KEY: weloverust AWS_ENDPOINT_URL: http://localhost:4566 - AWS_STORAGE_ALLOW_HTTP: "1" + AWS_ALLOW_HTTP: "1" AZURE_USE_EMULATOR: "1" AZURE_STORAGE_ALLOW_HTTP: "1" AZURITE_BLOB_STORAGE_URL: "http://localhost:10000" diff --git a/python/tests/conftest.py b/python/tests/conftest.py index 4cf4cf4e5e..6ddb68a526 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -103,7 +103,7 @@ def s3_localstack_creds(): @pytest.fixture() def s3_localstack(monkeypatch, s3_localstack_creds): - monkeypatch.setenv("AWS_STORAGE_ALLOW_HTTP", "TRUE") + monkeypatch.setenv("AWS_ALLOW_HTTP", "TRUE") for key, value in s3_localstack_creds.items(): monkeypatch.setenv(key, value) diff --git a/python/tests/test_fs.py b/python/tests/test_fs.py index 6516a62558..92e77e794a 100644 --- a/python/tests/test_fs.py +++ b/python/tests/test_fs.py @@ -139,7 +139,7 @@ def test_roundtrip_s3_direct(s3_localstack_creds, sample_data: pa.Table): # Can pass storage_options in directly storage_opts = { - "AWS_STORAGE_ALLOW_HTTP": "true", + "AWS_ALLOW_HTTP": "true", "AWS_S3_ALLOW_UNSAFE_RENAME": "true", } storage_opts.update(s3_localstack_creds) diff --git a/rust/src/builder.rs b/rust/src/builder.rs index ca7661800c..decfda8db0 100644 --- a/rust/src/builder.rs +++ b/rust/src/builder.rs @@ -341,7 +341,7 @@ pub mod s3_storage_options { /// See also . pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; /// Allow http connections - mainly useful for integration tests - pub const AWS_STORAGE_ALLOW_HTTP: &str = "AWS_STORAGE_ALLOW_HTTP"; + pub const AWS_ALLOW_HTTP: &str = "AWS_ALLOW_HTTP"; /// If set to "true", allows creating commits without concurrent writer protection. /// Only safe if there is one writer to a given table. diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index db6eac6039..15b3829138 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -116,7 +116,7 @@ impl StorageOptions { if let Ok(value) = std::env::var("AZURE_STORAGE_USE_HTTP") { options.insert("allow_http".into(), value); } - if let Ok(value) = std::env::var("AWS_STORAGE_ALLOW_HTTP") { + if let Ok(value) = std::env::var("AWS_ALLOW_HTTP") { options.insert("allow_http".into(), value); } Self(options) diff --git a/rust/tests/common/s3.rs b/rust/tests/common/s3.rs index cc8135e23f..1ba5d1cd15 100644 --- a/rust/tests/common/s3.rs +++ b/rust/tests/common/s3.rs @@ -21,7 +21,7 @@ pub async fn setup_s3_context() -> TestContext { env::set_var("AWS_ACCESS_KEY_ID", "deltalake"); env::set_var("AWS_SECRET_ACCESS_KEY", "weloverust"); env::set_var("AWS_DEFAULT_REGION", ®ion); - env::set_var("AWS_STORAGE_ALLOW_HTTP", "TRUE"); + env::set_var("AWS_ALLOW_HTTP", "TRUE"); cli.create_bucket(bucket_name, &endpoint); cli.create_table( @@ -40,7 +40,7 @@ pub async fn setup_s3_context() -> TestContext { config.insert("AWS_SECRET_ACCESS_KEY".to_owned(), "weloverust".to_owned()); config.insert("AWS_S3_LOCKING_PROVIDER".to_owned(), "dynamodb".to_owned()); config.insert("DYNAMO_LOCK_TABLE_NAME".to_owned(), lock_table.clone()); - config.insert("AWS_STORAGE_ALLOW_HTTP".to_owned(), "TRUE".to_string()); + config.insert("AWS_ALLOW_HTTP".to_owned(), "TRUE".to_string()); TestContext { config, diff --git a/rust/tests/integration_datafusion.rs b/rust/tests/integration_datafusion.rs index 2489ab9371..30a4345a6c 100644 --- a/rust/tests/integration_datafusion.rs +++ b/rust/tests/integration_datafusion.rs @@ -1040,7 +1040,7 @@ async fn simple_query(context: &IntegrationContext) -> TestResult { let dynamo_lock_option = "'DYNAMO_LOCK_OWNER_NAME' 's3::deltars/simple'".to_string(); let options = match context.integration { - StorageIntegration::Amazon => format!("'AWS_STORAGE_ALLOW_HTTP' '1', {dynamo_lock_option}"), + StorageIntegration::Amazon => format!("'AWS_ALLOW_HTTP' '1', {dynamo_lock_option}"), StorageIntegration::Microsoft => { format!("'AZURE_STORAGE_ALLOW_HTTP' '1', {dynamo_lock_option}") } From 7cb24ca90c6b0a7b217116f15857be60d8bab6e7 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 00:07:58 +0200 Subject: [PATCH 25/99] fix: feature gate azure update --- rust/src/storage/config.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index 15b3829138..d6b95c74aa 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -73,6 +73,7 @@ impl ObjectStoreScheme { } else if host.contains("dfs.fabric.microsoft.com") || host.contains("blob.fabric.microsoft.com") { + #[cfg(feature = "azure")] if !options .as_azure_options() .contains_key(&AzureConfigKey::UseFabricEndpoint) From 6fb62c074cd9314924b0cf2d45b80250dad4a4ad Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 00:10:31 +0200 Subject: [PATCH 26/99] feat: more robust azure config handling --- rust/examples/recordbatch-writer.rs | 2 +- rust/src/storage/config.rs | 73 ++++----- rust/src/storage/config/azure.rs | 226 ++++++++++++++++++++++++++++ 3 files changed, 256 insertions(+), 45 deletions(-) create mode 100644 rust/src/storage/config/azure.rs diff --git a/rust/examples/recordbatch-writer.rs b/rust/examples/recordbatch-writer.rs index cca9c6e3fc..8bafa3b2ac 100644 --- a/rust/examples/recordbatch-writer.rs +++ b/rust/examples/recordbatch-writer.rs @@ -45,7 +45,7 @@ async fn main() -> Result<(), DeltaTableError> { info!("It doesn't look like our delta table has been created"); create_initialized_table(&table_path).await } - Err(err) => Err(err).unwrap(), + Err(err) => panic!("{:?}", err), }; let writer_properties = WriterProperties::builder() diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index d6b95c74aa..9e0ad0d040 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -30,6 +30,8 @@ use object_store::gcp::GoogleConfigKey; ))] use std::str::FromStr; +mod azure; + /// Recognises various URL formats, identifying the relevant [`ObjectStore`](crate::ObjectStore) #[derive(Debug, Eq, PartialEq)] enum ObjectStoreScheme { @@ -53,7 +55,10 @@ impl ObjectStoreScheme { /// Create an [`ObjectStoreScheme`] from the provided [`Url`] /// /// Returns the [`ObjectStoreScheme`] and the remaining [`Path`] - fn parse(url: &Url, options: &mut StorageOptions) -> Result<(Self, Path), ObjectStoreError> { + fn parse( + url: &Url, + #[allow(unused)] options: &mut StorageOptions, + ) -> Result<(Self, Path), ObjectStoreError> { let strip_bucket = || Some(url.path().strip_prefix('/')?.split_once('/')?.1); let (scheme, path) = match (url.scheme(), url.host_str()) { @@ -129,8 +134,10 @@ impl StorageOptions { for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { if let Ok(config_key) = AzureConfigKey::from_str(&key.to_ascii_lowercase()) { - self.0 - .insert(config_key.as_ref().to_string(), value.to_string()); + if !self.0.contains_key(config_key.as_ref()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } } } } @@ -142,8 +149,10 @@ impl StorageOptions { for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { if let Ok(config_key) = GoogleConfigKey::from_str(&key.to_ascii_lowercase()) { - self.0 - .insert(config_key.as_ref().to_string(), value.to_string()); + if !self.0.contains_key(config_key.as_ref()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } } } } @@ -155,8 +164,10 @@ impl StorageOptions { for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { - self.0 - .insert(config_key.as_ref().to_string(), value.to_string()); + if !self.0.contains_key(config_key.as_ref()) { + self.0 + .insert(config_key.as_ref().to_string(), value.to_string()); + } } } } @@ -226,49 +237,23 @@ pub(crate) fn configure_store( )?)), #[cfg(any(feature = "s3", feature = "s3-native-tls"))] ObjectStoreScheme::AmazonS3 => { - if options.0.is_empty() { - options.with_env_s3(); - } - if let Ok((store, prefix)) = parse_url_opts(url, options.as_s3_options()) { - let store = S3StorageBackend::try_new( - Arc::new(store), - S3StorageOptions::from_map(&options.0), - )?; - Ok(Arc::new(PrefixStore::new(store, prefix))) - } else { - options.with_env_s3(); - let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; - let store = S3StorageBackend::try_new( - Arc::new(store), - S3StorageOptions::from_map(&options.0), - )?; - Ok(Arc::new(PrefixStore::new(store, prefix))) - } + options.with_env_s3(); + let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; + let store = + S3StorageBackend::try_new(Arc::new(store), S3StorageOptions::from_map(&options.0))?; + Ok(Arc::new(PrefixStore::new(store, prefix))) } #[cfg(feature = "azure")] ObjectStoreScheme::MicrosoftAzure => { - if options.0.is_empty() { - options.with_env_azure(); - } - if let Ok((store, prefix)) = parse_url_opts(url, options.as_azure_options()) { - Ok(Arc::new(PrefixStore::new(store, prefix))) - } else { - options.with_env_azure(); - let (store, prefix) = parse_url_opts(url, options.as_azure_options())?; - Ok(Arc::new(PrefixStore::new(store, prefix))) - } + let config = azure::AzureConfigHelper::try_new(options.as_azure_options())?.build()?; + let (store, prefix) = parse_url_opts(url, config)?; + Ok(Arc::new(PrefixStore::new(store, prefix))) } #[cfg(feature = "gcs")] ObjectStoreScheme::GoogleCloudStorage => { - if options.0.is_empty() { - options.with_env_gcs(); - } - if let Ok((store, prefix)) = parse_url_opts(url, options.as_gcs_options()) { - Ok(Arc::new(PrefixStore::new(store, prefix))) - } else { - let (store, prefix) = parse_url_opts(url, options.as_gcs_options())?; - Ok(Arc::new(PrefixStore::new(store, prefix))) - } + options.with_env_gcs(); + let (store, prefix) = parse_url_opts(url, options.as_gcs_options())?; + Ok(Arc::new(PrefixStore::new(store, prefix))) } #[cfg(feature = "hdfs")] ObjectStoreScheme::Hdfs => { diff --git a/rust/src/storage/config/azure.rs b/rust/src/storage/config/azure.rs new file mode 100644 index 0000000000..e556b9eb6e --- /dev/null +++ b/rust/src/storage/config/azure.rs @@ -0,0 +1,226 @@ +//! Auxiliary module for generating a valig Azure configuration. +//! +//! Azure offers many different ways to authenticate against storage accounts and +//! provide credentials for a service principal. Some of this configutaion may +//! partially be specified in the environment. This module establishes a structured +//! way how we discover valid credentials and some heuristics on how they are prioritized. +use std::collections::{hash_map::Entry, HashMap}; +use std::str::FromStr; + +use object_store::azure::AzureConfigKey; +use object_store::Error; + +use crate::{DeltaResult, DeltaTableError}; + +lazy_static::lazy_static! { + static ref CREDENTIAL_KEYS: Vec = + Vec::from_iter([ + AzureConfigKey::ClientId, + AzureConfigKey::ClientSecret, + AzureConfigKey::FederatedTokenFile, + AzureConfigKey::SasKey, + AzureConfigKey::Token, + AzureConfigKey::MsiEndpoint, + AzureConfigKey::ObjectId, + AzureConfigKey::MsiResourceId, + ]); +} + +/// Credential +enum AzureCredential { + /// Using the account master key + AccessKey, + /// Using a static bearer token + BearerToken, + /// Authorizing with secret + ClientSecret, + /// Using a shared access signature + ManagedIdentity, + /// Using a shared access signature + SasKey, + /// Using workload identity + WorkloadIdentity, +} + +impl FromStr for AzureCredential { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s { + "access_key" => Ok(AzureCredential::AccessKey), + "bearer_token" => Ok(AzureCredential::BearerToken), + "client_secret" => Ok(AzureCredential::ClientSecret), + "managed_identity" => Ok(AzureCredential::ManagedIdentity), + "workload_identity" => Ok(AzureCredential::WorkloadIdentity), + "sas_key" => Ok(AzureCredential::SasKey), + _ => Err(DeltaTableError::Generic(format!( + "Cannot parse AzureCredential variant from {}", + s + ))), + } + } +} + +impl AsRef for AzureCredential { + fn as_ref(&self) -> &str { + match self { + Self::AccessKey => "access_key", + Self::BearerToken => "bearer_token", + Self::ClientSecret => "client_secret", + Self::ManagedIdentity => "managed_identity", + Self::SasKey => "sas_key", + Self::WorkloadIdentity => "workload_identity", + } + } +} + +impl AzureCredential { + /// Reys required for config + fn keys(&self) -> Vec { + match self { + Self::AccessKey => Vec::from_iter([AzureConfigKey::AccessKey]), + Self::BearerToken => Vec::from_iter([AzureConfigKey::Token]), + Self::ClientSecret => Vec::from_iter([ + AzureConfigKey::ClientId, + AzureConfigKey::ClientSecret, + AzureConfigKey::AuthorityId, + ]), + Self::WorkloadIdentity => Vec::from_iter([ + AzureConfigKey::AuthorityId, + AzureConfigKey::ClientId, + AzureConfigKey::FederatedTokenFile, + ]), + Self::SasKey => Vec::from_iter([AzureConfigKey::SasKey]), + Self::ManagedIdentity => Vec::new(), + } + } +} + +/// Helper struct to create full configuration from passed options and environment +/// +/// Main concern is to pick the desired credential for connecting to starage backend +/// based on a provided configuration and configuration set in the environment. +pub(crate) struct AzureConfigHelper { + config: HashMap, + env_config: HashMap, + priority: Vec, +} + +impl AzureConfigHelper { + /// Create a new [`ConfigHelper`] + pub fn try_new( + config: impl IntoIterator, impl Into)>, + ) -> DeltaResult { + let mut env_config = HashMap::new(); + for (os_key, os_value) in std::env::vars_os() { + if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { + if key.starts_with("AZURE_") { + if let Ok(config_key) = AzureConfigKey::from_str(&key.to_ascii_lowercase()) { + env_config.insert(config_key, value.to_string()); + } + } + } + } + + Ok(Self { + config: config + .into_iter() + .map(|(key, value)| Ok((AzureConfigKey::from_str(key.as_ref())?, value.into()))) + .collect::>()?, + env_config, + priority: Vec::from_iter([ + AzureCredential::AccessKey, + AzureCredential::SasKey, + AzureCredential::BearerToken, + AzureCredential::ClientSecret, + AzureCredential::WorkloadIdentity, + ]), + }) + } + + /// Check if all credential keys are contained in passed config + fn has_full_config(&self, cred: &AzureCredential) -> bool { + cred.keys().iter().all(|key| self.config.contains_key(key)) + } + + /// Check if any credential keys are contained in passed config + fn has_any_config(&self, cred: &AzureCredential) -> bool { + cred.keys().iter().any(|key| self.config.contains_key(key)) + } + + /// Check if all credential keys can be provided using the env + fn has_full_config_with_env(&self, cred: &AzureCredential) -> bool { + cred.keys() + .iter() + .all(|key| self.config.contains_key(key) || self.env_config.contains_key(key)) + } + + /// Generate a cofiguration augmented with options from the environment + pub fn build(mut self) -> DeltaResult> { + let mut has_credential = false; + + if self.config.contains_key(&AzureConfigKey::UseAzureCli) { + has_credential = true; + } + + // try using only passed config options + if !has_credential { + for cred in &self.priority { + if self.has_full_config(cred) { + has_credential = true; + break; + } + } + } + + // try partially avaialbe credentials augmented by environment + if !has_credential { + for cred in &self.priority { + if self.has_any_config(cred) && self.has_full_config_with_env(cred) { + for key in cred.keys() { + if let Entry::Vacant(e) = self.config.entry(key) { + e.insert(self.env_config.get(&key).unwrap().to_owned()); + } + } + has_credential = true; + break; + } + } + } + + // try getting credentials only from the environment + if !has_credential { + for cred in &self.priority { + if self.has_full_config_with_env(cred) { + for key in cred.keys() { + if let Entry::Vacant(e) = self.config.entry(key) { + e.insert(self.env_config.get(&key).unwrap().to_owned()); + } + } + has_credential = true; + break; + } + } + } + + let omit_keys = if has_credential { + CREDENTIAL_KEYS.clone() + } else { + Vec::new() + }; + + // Add keys from the environment to the configuration, as e.g. client configuration options. + // NOTE We have to specifically configure omitting keys, since workload identity can + // work purely using defaults, but partial config may be present in the environment. + // Preference of conflicting configs (e.g. msi resource id vs. client id is handled in object store) + for key in self.env_config.keys() { + if !omit_keys.contains(key) { + if let Entry::Vacant(e) = self.config.entry(*key) { + e.insert(self.env_config.get(key).unwrap().to_owned()); + } + } + } + + Ok(self.config) + } +} From 21b94836253adde89f89b2bf25d4a003a3d5c36b Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 10:06:12 +0200 Subject: [PATCH 27/99] fix: in memory store handling --- rust/src/storage/config.rs | 37 ++++++++++++++++++---------- rust/tests/integration_datafusion.rs | 2 +- 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index 9e0ad0d040..e06e969adc 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -2,13 +2,14 @@ use std::collections::HashMap; use std::sync::Arc; -use object_store::local::LocalFileSystem; +use object_store::memory::InMemory; use object_store::path::Path; use object_store::prefix::PrefixStore; -use object_store::{parse_url_opts, DynObjectStore, Error as ObjectStoreError}; +use object_store::{parse_url_opts, DynObjectStore, Error as ObjectStoreError, ObjectStore}; use serde::{Deserialize, Serialize}; use url::Url; +use super::file::FileStorageBackend; use super::utils::str_is_truthy; use crate::errors::{DeltaResult, DeltaTableError}; @@ -30,6 +31,7 @@ use object_store::gcp::GoogleConfigKey; ))] use std::str::FromStr; +#[cfg(feature = "azure")] mod azure; /// Recognises various URL formats, identifying the relevant [`ObjectStore`](crate::ObjectStore) @@ -229,31 +231,32 @@ pub(crate) fn configure_store( ) -> DeltaResult> { let (scheme, _prefix) = ObjectStoreScheme::parse(url, options)?; match scheme { - ObjectStoreScheme::Local => Ok(Arc::new(LocalFileSystem::new_with_prefix( - url.to_file_path() - .map_err(|_| DeltaTableError::InvalidTableLocation(url.to_string()))? - .to_str() - .ok_or_else(|| DeltaTableError::InvalidTableLocation(url.to_string()))?, - )?)), + ObjectStoreScheme::Local => { + let path = url + .to_file_path() + .map_err(|_| DeltaTableError::InvalidTableLocation(url.to_string()))?; + Ok(Arc::new(FileStorageBackend::try_new(path)?)) + } + ObjectStoreScheme::Memory => url_prefix_handler(InMemory::new(), Path::parse(url.path())?), #[cfg(any(feature = "s3", feature = "s3-native-tls"))] ObjectStoreScheme::AmazonS3 => { options.with_env_s3(); let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; let store = S3StorageBackend::try_new(Arc::new(store), S3StorageOptions::from_map(&options.0))?; - Ok(Arc::new(PrefixStore::new(store, prefix))) + url_prefix_handler(store, prefix) } #[cfg(feature = "azure")] ObjectStoreScheme::MicrosoftAzure => { let config = azure::AzureConfigHelper::try_new(options.as_azure_options())?.build()?; let (store, prefix) = parse_url_opts(url, config)?; - Ok(Arc::new(PrefixStore::new(store, prefix))) + url_prefix_handler(store, prefix) } #[cfg(feature = "gcs")] ObjectStoreScheme::GoogleCloudStorage => { options.with_env_gcs(); let (store, prefix) = parse_url_opts(url, options.as_gcs_options())?; - Ok(Arc::new(PrefixStore::new(store, prefix))) + url_prefix_handler(store, prefix) } #[cfg(feature = "hdfs")] ObjectStoreScheme::Hdfs => { @@ -263,7 +266,7 @@ pub(crate) fn configure_store( url.as_ref() )) })?; - Ok(Arc::new(PrefixStore::new(store, _prefix))) + url_prefix_handler(store, _prefix) } #[cfg(not(feature = "hdfs"))] ObjectStoreScheme::Hdfs => Err(DeltaTableError::MissingFeature { @@ -272,11 +275,19 @@ pub(crate) fn configure_store( }), _ => { let (store, prefix) = parse_url_opts(url, options.0.clone())?; - Ok(Arc::new(PrefixStore::new(store, prefix))) + url_prefix_handler(store, prefix) } } } +fn url_prefix_handler(store: T, prefix: Path) -> DeltaResult> { + if prefix != Path::from("/") { + Ok(Arc::new(PrefixStore::new(store, prefix))) + } else { + Ok(Arc::new(store)) + } +} + #[cfg(test)] mod test { use crate::ensure_table_uri; diff --git a/rust/tests/integration_datafusion.rs b/rust/tests/integration_datafusion.rs index 30a4345a6c..68c9149ac7 100644 --- a/rust/tests/integration_datafusion.rs +++ b/rust/tests/integration_datafusion.rs @@ -202,7 +202,7 @@ mod local { // Create target Delta Table let target_table = CreateBuilder::new() - .with_location("memory://target") + .with_location("memory:///target") .with_columns(fields) .with_table_name("target") .await?; From 05e6444e2bf34177cfe8d46f14ee159e4cb66f4f Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 23 Sep 2023 14:08:08 +0200 Subject: [PATCH 28/99] feat: use object-store's s3 store if copy-if-not-exists headers are specified (#1356) --- rust/src/storage/config.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index e06e969adc..76c2e6dd81 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -242,9 +242,18 @@ pub(crate) fn configure_store( ObjectStoreScheme::AmazonS3 => { options.with_env_s3(); let (store, prefix) = parse_url_opts(url, options.as_s3_options())?; - let store = - S3StorageBackend::try_new(Arc::new(store), S3StorageOptions::from_map(&options.0))?; - url_prefix_handler(store, prefix) + if options + .as_s3_options() + .contains_key(&AmazonS3ConfigKey::CopyIfNotExists) + { + url_prefix_handler(store, prefix) + } else { + let store = S3StorageBackend::try_new( + Arc::new(store), + S3StorageOptions::from_map(&options.0), + )?; + url_prefix_handler(store, prefix) + } } #[cfg(feature = "azure")] ObjectStoreScheme::MicrosoftAzure => { From 65179b6fd0180b62cea8ecdd4ce16918fb6138d3 Mon Sep 17 00:00:00 2001 From: Robert Pack <42610831+roeap@users.noreply.github.com> Date: Tue, 26 Sep 2023 03:17:32 +0200 Subject: [PATCH 29/99] refactor: re-organize top level modules (#1434) # Description ~This contains changes from #1432, will rebase once that's merged.~ This PR constitutes the bulk of re-organising our top level modules. - move `DeltaTable*` structs into new `table` module - move table configuration into `table` module - move schema related modules into `schema` module - rename `action` module to `protocol` - hoping to isolate everything that can one day be the log kernel. ~It also removes the deprecated commit logic from `DeltaTable` and updates call sites and tests accordingly.~ I am planning one more follow up, where I hope to make `transactions` currently within `operations` a top level module. While the number of touched files here is already massive, I want to do this in a follow up, as it will also include some updates to the transactions itself, that should be more carefully reviewed. # Related Issue(s) closes: #1136 # Documentation --- python/src/error.rs | 2 +- python/src/lib.rs | 18 ++-- rust/benches/read_checkpoint.rs | 4 +- rust/examples/basic_operations.rs | 2 +- rust/src/data_catalog/storage/mod.rs | 3 +- rust/src/delta_datafusion.rs | 12 +-- rust/src/errors.rs | 2 +- rust/src/lib.rs | 94 ++++++++++++++----- rust/src/operations/create.rs | 13 +-- rust/src/operations/delete.rs | 11 ++- rust/src/operations/filesystem_check.rs | 4 +- rust/src/operations/load.rs | 2 +- rust/src/operations/merge.rs | 33 +++---- rust/src/operations/mod.rs | 4 +- rust/src/operations/optimize.rs | 6 +- rust/src/operations/restore.rs | 12 +-- .../transaction/conflict_checker.rs | 37 +++++--- rust/src/operations/transaction/mod.rs | 4 +- rust/src/operations/transaction/state.rs | 4 +- rust/src/operations/transaction/test_utils.rs | 12 +-- rust/src/operations/update.rs | 6 +- rust/src/operations/vacuum.rs | 2 +- rust/src/operations/write.rs | 8 +- rust/src/operations/writer.rs | 23 ++--- rust/src/{action => protocol}/checkpoints.rs | 11 ++- rust/src/{action => protocol}/mod.rs | 6 +- .../parquet2_read/boolean.rs | 2 +- .../parquet2_read/dictionary/binary.rs | 0 .../parquet2_read/dictionary/mod.rs | 0 .../parquet2_read/dictionary/primitive.rs | 0 .../{action => protocol}/parquet2_read/map.rs | 2 +- .../{action => protocol}/parquet2_read/mod.rs | 6 +- .../parquet2_read/primitive.rs | 2 +- .../parquet2_read/stats.rs | 2 +- .../parquet2_read/string.rs | 2 +- .../parquet2_read/validity.rs | 0 .../{action => protocol}/parquet_read/mod.rs | 2 +- rust/src/{action => protocol}/serde_path.rs | 0 rust/src/{ => protocol}/time_utils.rs | 0 .../arrow_convert.rs} | 0 rust/src/{schema.rs => schema/mod.rs} | 4 + rust/src/{ => schema}/partitions.rs | 54 +++++------ rust/src/storage/config.rs | 2 +- rust/src/storage/mod.rs | 2 +- rust/src/storage/s3.rs | 2 +- rust/src/storage/utils.rs | 6 +- rust/src/{ => table}/builder.rs | 2 +- rust/src/{delta_config.rs => table/config.rs} | 2 +- rust/src/{delta.rs => table/mod.rs} | 89 ++++-------------- rust/src/{table_state.rs => table/state.rs} | 53 ++++++----- .../state_arrow.rs} | 6 +- rust/src/test_utils.rs | 2 +- rust/src/writer/json.rs | 22 +++-- rust/src/writer/mod.rs | 2 +- rust/src/writer/record_batch.rs | 31 +----- rust/src/writer/stats.rs | 6 +- rust/src/writer/test_utils.rs | 8 +- rust/tests/checkpoint_writer.rs | 10 +- rust/tests/command_filesystem_check.rs | 14 +-- rust/tests/command_optimize.rs | 2 +- rust/tests/command_restore.rs | 2 +- rust/tests/commit_info_format.rs | 2 +- rust/tests/common/mod.rs | 6 +- rust/tests/fs_common/mod.rs | 2 +- rust/tests/integration_concurrent_writes.rs | 6 +- rust/tests/integration_datafusion.rs | 14 +-- rust/tests/integration_object_store.rs | 4 +- rust/tests/integration_read.rs | 8 +- rust/tests/read_delta_log_test.rs | 2 +- rust/tests/read_delta_partitions_test.rs | 2 +- rust/tests/repair_s3_rename_test.rs | 8 +- 71 files changed, 359 insertions(+), 369 deletions(-) rename rust/src/{action => protocol}/checkpoints.rs (98%) rename rust/src/{action => protocol}/mod.rs (99%) rename rust/src/{action => protocol}/parquet2_read/boolean.rs (98%) rename rust/src/{action => protocol}/parquet2_read/dictionary/binary.rs (100%) rename rust/src/{action => protocol}/parquet2_read/dictionary/mod.rs (100%) rename rust/src/{action => protocol}/parquet2_read/dictionary/primitive.rs (100%) rename rust/src/{action => protocol}/parquet2_read/map.rs (99%) rename rust/src/{action => protocol}/parquet2_read/mod.rs (99%) rename rust/src/{action => protocol}/parquet2_read/primitive.rs (99%) rename rust/src/{action => protocol}/parquet2_read/stats.rs (85%) rename rust/src/{action => protocol}/parquet2_read/string.rs (99%) rename rust/src/{action => protocol}/parquet2_read/validity.rs (100%) rename rust/src/{action => protocol}/parquet_read/mod.rs (99%) rename rust/src/{action => protocol}/serde_path.rs (100%) rename rust/src/{ => protocol}/time_utils.rs (100%) rename rust/src/{delta_arrow.rs => schema/arrow_convert.rs} (100%) rename rust/src/{schema.rs => schema/mod.rs} (99%) rename rust/src/{ => schema}/partitions.rs (90%) rename rust/src/{ => table}/builder.rs (99%) rename rust/src/{delta_config.rs => table/config.rs} (99%) rename rust/src/{delta.rs => table/mod.rs} (92%) rename rust/src/{table_state.rs => table/state.rs} (91%) rename rust/src/{table_state_arrow.rs => table/state_arrow.rs} (99%) diff --git a/python/src/error.rs b/python/src/error.rs index 229f7c83dc..2422dfc441 100644 --- a/python/src/error.rs +++ b/python/src/error.rs @@ -1,5 +1,5 @@ use arrow_schema::ArrowError; -use deltalake::action::ProtocolError; +use deltalake::protocol::ProtocolError; use deltalake::{errors::DeltaTableError, ObjectStoreError}; use pyo3::exceptions::{ PyException, PyFileNotFoundError, PyIOError, PyNotImplementedError, PyValueError, diff --git a/python/src/lib.rs b/python/src/lib.rs index 8d7cdb486d..8115c1bb76 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -14,13 +14,9 @@ use std::time::{SystemTime, UNIX_EPOCH}; use arrow::pyarrow::PyArrowType; use chrono::{DateTime, Duration, FixedOffset, Utc}; -use deltalake::action::{ - self, Action, ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats, -}; use deltalake::arrow::compute::concat_batches; use deltalake::arrow::record_batch::RecordBatch; use deltalake::arrow::{self, datatypes::Schema as ArrowSchema}; -use deltalake::builder::DeltaTableBuilder; use deltalake::checkpoints::create_checkpoint; use deltalake::datafusion::prelude::SessionContext; use deltalake::delta_datafusion::DeltaDataChecker; @@ -30,6 +26,10 @@ use deltalake::operations::restore::RestoreBuilder; use deltalake::operations::transaction::commit; use deltalake::operations::vacuum::VacuumBuilder; use deltalake::partitions::PartitionFilter; +use deltalake::protocol::{ + self, Action, ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats, +}; +use deltalake::DeltaTableBuilder; use deltalake::{DeltaOps, Invariant, Schema}; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; use pyo3::prelude::*; @@ -497,7 +497,7 @@ impl RawDeltaTable { let existing_schema = self._table.get_schema().map_err(PythonError::from)?; - let mut actions: Vec = add_actions + let mut actions: Vec = add_actions .iter() .map(|add| Action::add(add.into())) .collect(); @@ -515,7 +515,7 @@ impl RawDeltaTable { .map_err(PythonError::from)?; for old_add in add_actions { - let remove_action = Action::remove(action::Remove { + let remove_action = Action::remove(protocol::Remove { path: old_add.path.clone(), deletion_timestamp: Some(current_timestamp()), data_change: true, @@ -536,7 +536,7 @@ impl RawDeltaTable { .map_err(PythonError::from)? .clone(); metadata.schema = schema; - let metadata_action = action::MetaData::try_from(metadata) + let metadata_action = protocol::MetaData::try_from(metadata) .map_err(|_| PyValueError::new_err("Failed to reparse metadata"))?; actions.push(Action::metaData(metadata_action)); } @@ -795,9 +795,9 @@ pub struct PyAddAction { stats: Option, } -impl From<&PyAddAction> for action::Add { +impl From<&PyAddAction> for protocol::Add { fn from(action: &PyAddAction) -> Self { - action::Add { + protocol::Add { path: action.path.clone(), size: action.size, partition_values: action.partition_values.clone(), diff --git a/rust/benches/read_checkpoint.rs b/rust/benches/read_checkpoint.rs index 9824f15eb0..2ecbee661b 100644 --- a/rust/benches/read_checkpoint.rs +++ b/rust/benches/read_checkpoint.rs @@ -1,6 +1,6 @@ use criterion::{criterion_group, criterion_main, Criterion}; -use deltalake::delta::DeltaTableConfig; -use deltalake::table_state::DeltaTableState; +use deltalake::table::state::DeltaTableState; +use deltalake::DeltaTableConfig; use std::fs::File; use std::io::Read; diff --git a/rust/examples/basic_operations.rs b/rust/examples/basic_operations.rs index d95aadfb78..0732791b74 100644 --- a/rust/examples/basic_operations.rs +++ b/rust/examples/basic_operations.rs @@ -4,7 +4,7 @@ use arrow::{ record_batch::RecordBatch, }; use deltalake::operations::collect_sendable_stream; -use deltalake::{action::SaveMode, DeltaOps, SchemaDataType, SchemaField}; +use deltalake::{protocol::SaveMode, DeltaOps, SchemaDataType, SchemaField}; use parquet::{ basic::{Compression, ZstdLevel}, file::properties::WriterProperties, diff --git a/rust/src/data_catalog/storage/mod.rs b/rust/src/data_catalog/storage/mod.rs index 90d9db8fdb..f645d370c1 100644 --- a/rust/src/data_catalog/storage/mod.rs +++ b/rust/src/data_catalog/storage/mod.rs @@ -13,8 +13,9 @@ use futures::TryStreamExt; use object_store::ObjectStore; use crate::errors::DeltaResult; +use crate::open_table_with_storage_options; use crate::storage::config::{configure_store, StorageOptions}; -use crate::{ensure_table_uri, open_table_with_storage_options}; +use crate::table::builder::ensure_table_uri; const DELTA_LOG_FOLDER: &str = "_delta_log"; diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index e69c7abbfe..e542413cfd 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -67,11 +67,11 @@ use datafusion_proto::physical_plan::PhysicalExtensionCodec; use object_store::ObjectMeta; use url::Url; -use crate::action::{self, Add}; -use crate::builder::ensure_table_uri; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{self, Add}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; +use crate::table::builder::ensure_table_uri; +use crate::table::state::DeltaTableState; use crate::{open_table, open_table_with_storage_options, DeltaTable, Invariant, SchemaDataType}; const PATH_COLUMN: &str = "__delta_rs_path"; @@ -124,7 +124,7 @@ impl DeltaTableState { |acc, action| { let new_stats = action .get_stats() - .unwrap_or_else(|_| Some(action::Stats::default()))?; + .unwrap_or_else(|_| Some(protocol::Stats::default()))?; Some(Statistics { num_rows: acc .num_rows @@ -631,7 +631,7 @@ pub(crate) fn get_null_of_arrow_type(t: &ArrowDataType) -> DeltaResult PartitionedFile { let partition_values = schema @@ -1541,7 +1541,7 @@ mod tests { let mut partition_values = std::collections::HashMap::new(); partition_values.insert("month".to_string(), Some("1".to_string())); partition_values.insert("year".to_string(), Some("2015".to_string())); - let action = action::Add { + let action = protocol::Add { path: "year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string(), size: 10644, partition_values, diff --git a/rust/src/errors.rs b/rust/src/errors.rs index fed0e823f8..24989b2814 100644 --- a/rust/src/errors.rs +++ b/rust/src/errors.rs @@ -1,8 +1,8 @@ //! Exceptions for the deltalake crate use object_store::Error as ObjectStoreError; -use crate::action::ProtocolError; use crate::operations::transaction::TransactionError; +use crate::protocol::ProtocolError; /// A result returned by delta-rs pub type DeltaResult = Result; diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 2d03651cb9..af692fd5c9 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -82,42 +82,34 @@ compile_error!( "Features s3 and s3-native-tls are mutually exclusive and cannot be enabled together" ); -pub mod action; -pub mod builder; pub mod data_catalog; -pub mod delta; -pub mod delta_config; pub mod errors; pub mod operations; -pub mod partitions; +pub mod protocol; pub mod schema; pub mod storage; -pub mod table_state; -pub mod time_utils; +pub mod table; -#[cfg(feature = "arrow")] -pub mod table_state_arrow; - -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod delta_arrow; #[cfg(feature = "datafusion")] pub mod delta_datafusion; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; -pub use self::builder::*; +use std::collections::HashMap; + pub use self::data_catalog::{get_data_catalog, DataCatalog, DataCatalogError}; -pub use self::delta::*; -pub use self::delta_config::*; -pub use self::partitions::*; +pub use self::errors::*; +pub use self::schema::partitions::*; pub use self::schema::*; -pub use errors::*; +pub use self::table::builder::{ + DeltaTableBuilder, DeltaTableConfig, DeltaTableLoadOptions, DeltaVersion, +}; +pub use self::table::config::DeltaConfigKey; +pub use self::table::DeltaTable; pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; pub use operations::DeltaOps; // convenience exports for consumers to avoid aligning crate versions -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub use action::checkpoints; #[cfg(feature = "arrow")] pub use arrow; #[cfg(feature = "datafusion")] @@ -126,15 +118,70 @@ pub use datafusion; pub use parquet; #[cfg(feature = "parquet2")] pub use parquet2; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub use protocol::checkpoints; // needed only for integration tests // TODO can / should we move this into the test crate? #[cfg(feature = "integration_test")] pub mod test_utils; +/// Creates and loads a DeltaTable from the given path with current metadata. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table(table_uri: impl AsRef) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri).load().await?; + Ok(table) +} + +/// Same as `open_table`, but also accepts storage options to aid in building the table for a deduced +/// `StorageService`. +pub async fn open_table_with_storage_options( + table_uri: impl AsRef, + storage_options: HashMap, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_storage_options(storage_options) + .load() + .await?; + Ok(table) +} + +/// Creates a DeltaTable from the given path and loads it with the metadata from the given version. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table_with_version( + table_uri: impl AsRef, + version: i64, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_version(version) + .load() + .await?; + Ok(table) +} + +/// Creates a DeltaTable from the given path. +/// Loads metadata from the version appropriate based on the given ISO-8601/RFC-3339 timestamp. +/// Infers the storage backend to use from the scheme in the given table path. +pub async fn open_table_with_ds( + table_uri: impl AsRef, + ds: impl AsRef, +) -> Result { + let table = DeltaTableBuilder::from_uri(table_uri) + .with_datestring(ds)? + .load() + .await?; + Ok(table) +} + +/// Returns rust crate version, can be use used in language bindings to expose Rust core version +pub fn crate_version() -> &'static str { + env!("CARGO_PKG_VERSION") +} + #[cfg(test)] mod tests { use super::*; + use crate::table::PeekCommit; use std::collections::HashMap; #[tokio::test] @@ -153,7 +200,7 @@ mod tests { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 4); - assert!(tombstones.contains(&crate::action::Remove { + assert!(tombstones.contains(&crate::protocol::Remove { path: "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1564524298213), data_change: false, @@ -255,7 +302,7 @@ mod tests { ); let tombstones = table.get_state().all_tombstones(); assert_eq!(tombstones.len(), 1); - assert!(tombstones.contains(&crate::action::Remove { + assert!(tombstones.contains(&crate::protocol::Remove { path: "part-00001-911a94a2-43f6-4acb-8620-5e68c2654989-c000.snappy.parquet".to_string(), deletion_timestamp: Some(1615043776198), data_change: true, @@ -475,7 +522,10 @@ mod tests { let table_from_struct_stats = crate::open_table(table_uri).await.unwrap(); let table_from_json_stats = crate::open_table_with_version(table_uri, 1).await.unwrap(); - fn get_stats_for_file(table: &crate::DeltaTable, file_name: &str) -> crate::action::Stats { + fn get_stats_for_file( + table: &crate::DeltaTable, + file_name: &str, + ) -> crate::protocol::Stats { table .get_file_uris() .zip(table.get_stats()) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 697ab3ef1d..694d95a74e 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -9,13 +9,14 @@ use serde_json::{Map, Value}; use super::transaction::commit; use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; -use crate::action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; -use crate::builder::ensure_table_uri; -use crate::delta_config::DeltaConfigKey; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; use crate::storage::DeltaObjectStore; -use crate::{DeltaTable, DeltaTableBuilder, DeltaTableMetaData}; +use crate::table::builder::ensure_table_uri; +use crate::table::config::DeltaConfigKey; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder}; #[derive(thiserror::Error, Debug)] enum CreateError { @@ -148,7 +149,7 @@ impl CreateBuilder { /// /// Options may be passed in the HashMap or set as environment variables. /// - /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. + /// [crate::table::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. /// /// If an object store is also passed using `with_object_store()` these options will be ignored. @@ -322,8 +323,8 @@ impl std::future::IntoFuture for CreateBuilder { #[cfg(all(test, feature = "parquet"))] mod tests { use super::*; - use crate::delta_config::DeltaConfigKey; use crate::operations::DeltaOps; + use crate::table::config::DeltaConfigKey; use crate::writer::test_utils::get_delta_schema; use tempdir::TempDir; diff --git a/rust/src/operations/delete.rs b/rust/src/operations/delete.rs index 39b98cce24..d7f908680d 100644 --- a/rust/src/operations/delete.rs +++ b/rust/src/operations/delete.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use std::time::{Instant, SystemTime, UNIX_EPOCH}; -use crate::action::{Action, Add, Remove}; +use crate::protocol::{Action, Add, Remove}; use datafusion::execution::context::{SessionContext, SessionState}; use datafusion::physical_expr::create_physical_expr; use datafusion::physical_plan::filter::FilterExec; @@ -33,13 +33,14 @@ use parquet::file::properties::WriterProperties; use serde_json::Map; use serde_json::Value; -use crate::action::DeltaOperation; -use crate::delta_datafusion::{find_files, parquet_scan_from_actions, register_store}; +use crate::delta_datafusion::find_files; +use crate::delta_datafusion::{parquet_scan_from_actions, register_store}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; use crate::operations::write::write_execution_plan; +use crate::protocol::DeltaOperation; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; use super::datafusion_utils::Expression; @@ -324,8 +325,8 @@ impl std::future::IntoFuture for DeleteBuilder { #[cfg(test)] mod tests { - use crate::action::*; use crate::operations::DeltaOps; + use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::{get_arrow_schema, get_delta_schema}; use crate::DeltaTable; diff --git a/rust/src/operations/filesystem_check.rs b/rust/src/operations/filesystem_check.rs index 6cdf5390bf..bf047c45c4 100644 --- a/rust/src/operations/filesystem_check.rs +++ b/rust/src/operations/filesystem_check.rs @@ -24,11 +24,11 @@ pub use object_store::path::Path; use object_store::ObjectStore; use url::{ParseError, Url}; -use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::operations::transaction::commit; +use crate::protocol::{Action, Add, DeltaOperation, Remove}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; /// Audit the Delta Table's active files with the underlying file system. diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs index 9501c18011..7baa59e3e1 100644 --- a/rust/src/operations/load.rs +++ b/rust/src/operations/load.rs @@ -8,7 +8,7 @@ use futures::future::BoxFuture; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; #[derive(Debug, Clone)] diff --git a/rust/src/operations/merge.rs b/rust/src/operations/merge.rs index 29c5ffaa27..d088fbd3b7 100644 --- a/rust/src/operations/merge.rs +++ b/rust/src/operations/merge.rs @@ -32,6 +32,10 @@ //! .await? //! ```` +use std::collections::HashMap; +use std::sync::Arc; +use std::time::{Instant, SystemTime, UNIX_EPOCH}; + use arrow_schema::SchemaRef; use datafusion::error::Result as DataFusionResult; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -55,27 +59,16 @@ use datafusion_physical_expr::{create_physical_expr, expressions, PhysicalExpr}; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; use serde_json::{Map, Value}; -use std::{ - collections::HashMap, - sync::Arc, - time::{Instant, SystemTime, UNIX_EPOCH}, -}; -use crate::action::MergePredicate; +use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; +use super::transaction::commit; +use crate::delta_datafusion::{parquet_scan_from_actions, register_store}; use crate::operations::datafusion_utils::MetricObserverExec; -use crate::{ - action::{Action, DeltaOperation, Remove}, - delta_datafusion::{parquet_scan_from_actions, register_store}, - operations::write::write_execution_plan, - storage::{DeltaObjectStore, ObjectStoreRef}, - table_state::DeltaTableState, - DeltaResult, DeltaTable, DeltaTableError, -}; - -use super::{ - datafusion_utils::{into_expr, maybe_into_expr, Expression}, - transaction::commit, -}; +use crate::operations::write::write_execution_plan; +use crate::protocol::{Action, DeltaOperation, MergePredicate, Remove}; +use crate::storage::{DeltaObjectStore, ObjectStoreRef}; +use crate::table::state::DeltaTableState; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; const OPERATION_COLUMN: &str = "__delta_rs_operation"; const DELETE_COLUMN: &str = "__delta_rs_delete"; @@ -1105,8 +1098,8 @@ impl std::future::IntoFuture for MergeBuilder { #[cfg(test)] mod tests { - use crate::action::*; use crate::operations::DeltaOps; + use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::get_arrow_schema; use crate::writer::test_utils::get_delta_schema; diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 28dea06777..7b6cb27ace 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -10,8 +10,8 @@ use self::create::CreateBuilder; use self::filesystem_check::FileSystemCheckBuilder; use self::vacuum::VacuumBuilder; -use crate::builder::DeltaTableBuilder; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::table::builder::DeltaTableBuilder; use crate::DeltaTable; pub mod create; @@ -213,7 +213,7 @@ mod datafusion_utils { use datafusion_expr::Expr; use futures::{Stream, StreamExt}; - use crate::{table_state::DeltaTableState, DeltaResult}; + use crate::{table::state::DeltaTableState, DeltaResult}; /// Used to represent user input of either a Datafusion expression or string expression pub enum Expression { diff --git a/rust/src/operations/optimize.rs b/rust/src/operations/optimize.rs index c26150f9f3..eafc768519 100644 --- a/rust/src/operations/optimize.rs +++ b/rust/src/operations/optimize.rs @@ -41,10 +41,10 @@ use serde_json::Map; use super::transaction::commit; use super::writer::{PartitionWriter, PartitionWriterConfig}; -use crate::action::{self, Action, DeltaOperation}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{self, Action, DeltaOperation}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; use crate::{crate_version, DeltaTable, ObjectMeta, PartitionFilter}; @@ -311,7 +311,7 @@ fn create_remove( let deletion_time = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); let deletion_time = deletion_time.as_millis() as i64; - Ok(Action::remove(action::Remove { + Ok(Action::remove(protocol::Remove { path: path.to_string(), deletion_timestamp: Some(deletion_time), data_change: false, diff --git a/rust/src/operations/restore.rs b/rust/src/operations/restore.rs index 5800edd96f..c450554fea 100644 --- a/rust/src/operations/restore.rs +++ b/rust/src/operations/restore.rs @@ -30,11 +30,11 @@ use object_store::path::Path; use object_store::ObjectStore; use serde::Serialize; -use crate::action::{Action, Add, DeltaOperation, Remove}; use crate::operations::transaction::{prepare_commit, try_commit_transaction, TransactionError}; +use crate::protocol::{Action, Add, DeltaOperation, Protocol, Remove}; use crate::storage::ObjectStoreRef; -use crate::table_state::DeltaTableState; -use crate::{action, DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; +use crate::table::state::DeltaTableState; +use crate::{DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; /// Errors that can occur during restore #[derive(thiserror::Error, Debug)] @@ -202,12 +202,12 @@ async fn execute( let mut actions = vec![]; let protocol = if protocol_downgrade_allowed { - action::Protocol { + Protocol { min_reader_version: table.get_min_reader_version(), min_writer_version: table.get_min_writer_version(), } } else { - action::Protocol { + Protocol { min_reader_version: max( table.get_min_reader_version(), snapshot.min_reader_version(), @@ -249,7 +249,7 @@ async fn execute( async fn check_files_available( object_store: &dyn ObjectStore, - files: &Vec, + files: &Vec, ) -> DeltaResult<()> { for file in files { let file_path = Path::parse(file.path.clone())?; diff --git a/rust/src/operations/transaction/conflict_checker.rs b/rust/src/operations/transaction/conflict_checker.rs index 1058400787..d75e401def 100644 --- a/rust/src/operations/transaction/conflict_checker.rs +++ b/rust/src/operations/transaction/conflict_checker.rs @@ -5,11 +5,11 @@ use std::io::{BufRead, BufReader, Cursor}; use object_store::ObjectStore; use super::CommitInfo; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; -use crate::delta_config::IsolationLevel; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, Add, DeltaOperation, MetaData, Protocol, Remove}; use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; +use crate::table::config::IsolationLevel; +use crate::table::state::DeltaTableState; #[cfg(feature = "datafusion")] use super::state::AddContainer; @@ -53,8 +53,8 @@ pub enum CommitConflictError { /// you may need to upgrade your Delta Lake version. /// - When multiple writers are creating or replacing a table at the same time. /// - When multiple writers are writing to an empty path at the same time. - #[error("Protocol changed since last commit.")] - ProtocolChanged, + #[error("Protocol changed since last commit: {0}")] + ProtocolChanged(String), /// Error returned when the table requires an unsupported writer version #[error("Delta-rs does not support writer version {0}")] @@ -392,10 +392,18 @@ impl<'a> ConflictChecker<'a> { /// to read and write against the protocol set by the committed transaction. fn check_protocol_compatibility(&self) -> Result<(), CommitConflictError> { for p in self.winning_commit_summary.protocol() { - if self.txn_info.read_snapshot.min_reader_version() < p.min_reader_version - || self.txn_info.read_snapshot.min_writer_version() < p.min_writer_version - { - return Err(CommitConflictError::ProtocolChanged); + let (win_read, curr_read) = ( + p.min_reader_version, + self.txn_info.read_snapshot.min_reader_version(), + ); + let (win_write, curr_write) = ( + p.min_writer_version, + self.txn_info.read_snapshot.min_writer_version(), + ); + if curr_read < win_read || win_write < curr_write { + return Err(CommitConflictError::ProtocolChanged( + format!("reqired read/write {win_read}/{win_write}, current read/write {curr_read}/{curr_write}"), + )); }; } if !self.winning_commit_summary.protocol().is_empty() @@ -405,7 +413,9 @@ impl<'a> ConflictChecker<'a> { .iter() .any(|a| matches!(a, Action::protocol(_))) { - return Err(CommitConflictError::ProtocolChanged); + return Err(CommitConflictError::ProtocolChanged( + "protocol changed".into(), + )); }; Ok(()) } @@ -631,7 +641,7 @@ mod tests { use super::super::test_utils as tu; use super::super::test_utils::init_table_actions; use super::*; - use crate::action::Action; + use crate::protocol::Action; #[cfg(feature = "datafusion")] use datafusion_expr::{col, lit}; use serde_json::json; @@ -818,7 +828,10 @@ mod tests { vec![tu::create_protocol_action(None, None)], false, ); - assert!(matches!(result, Err(CommitConflictError::ProtocolChanged))); + assert!(matches!( + result, + Err(CommitConflictError::ProtocolChanged(_)) + )); // taint whole table // `read_whole_table` should disallow any concurrent change, even if the change diff --git a/rust/src/operations/transaction/mod.rs b/rust/src/operations/transaction/mod.rs index fe99af1749..cc26e75fc2 100644 --- a/rust/src/operations/transaction/mod.rs +++ b/rust/src/operations/transaction/mod.rs @@ -5,11 +5,11 @@ use object_store::path::Path; use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::{Map, Value}; -use crate::action::{Action, CommitInfo, DeltaOperation}; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, CommitInfo, DeltaOperation}; use crate::storage::commit_uri_from_version; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; mod conflict_checker; #[cfg(feature = "datafusion")] diff --git a/rust/src/operations/transaction/state.rs b/rust/src/operations/transaction/state.rs index 98a20d8866..6fe1d65aee 100644 --- a/rust/src/operations/transaction/state.rs +++ b/rust/src/operations/transaction/state.rs @@ -19,12 +19,12 @@ use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; use sqlparser::tokenizer::Tokenizer; -use crate::action::Add; use crate::delta_datafusion::{ get_null_of_arrow_type, logical_expr_to_physical_expr, to_correct_scalar_value, }; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::table_state::DeltaTableState; +use crate::protocol::Add; +use crate::table::state::DeltaTableState; impl DeltaTableState { /// Get the table schema as an [`ArrowSchemaRef`] diff --git a/rust/src/operations/transaction/test_utils.rs b/rust/src/operations/transaction/test_utils.rs index 92d981d5e1..cdd98f8d1f 100644 --- a/rust/src/operations/transaction/test_utils.rs +++ b/rust/src/operations/transaction/test_utils.rs @@ -1,12 +1,12 @@ #![allow(unused)] -use super::{prepare_commit, try_commit_transaction, CommitInfo}; -use crate::action::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; -use crate::table_state::DeltaTableState; -use crate::{ - DeltaTable, DeltaTableBuilder, DeltaTableMetaData, Schema, SchemaDataType, SchemaField, -}; use std::collections::HashMap; +use super::{prepare_commit, try_commit_transaction, CommitInfo}; +use crate::protocol::{Action, Add, DeltaOperation, MetaData, Protocol, Remove, SaveMode}; +use crate::table::state::DeltaTableState; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; + pub fn create_add_action( path: impl Into, data_change: bool, diff --git a/rust/src/operations/update.rs b/rust/src/operations/update.rs index a104732a8f..b030bc5644 100644 --- a/rust/src/operations/update.rs +++ b/rust/src/operations/update.rs @@ -43,10 +43,10 @@ use parquet::file::properties::WriterProperties; use serde_json::{Map, Value}; use crate::{ - action::{Action, DeltaOperation, Remove}, delta_datafusion::{find_files, parquet_scan_from_actions, register_store}, + protocol::{Action, DeltaOperation, Remove}, storage::{DeltaObjectStore, ObjectStoreRef}, - table_state::DeltaTableState, + table::state::DeltaTableState, DeltaResult, DeltaTable, DeltaTableError, }; @@ -475,7 +475,7 @@ mod tests { use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::{get_arrow_schema, get_delta_schema}; use crate::DeltaTable; - use crate::{action::*, DeltaResult}; + use crate::{protocol::SaveMode, DeltaResult}; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; use arrow_array::Int32Array; diff --git a/rust/src/operations/vacuum.rs b/rust/src/operations/vacuum.rs index 40e7657edf..ccaa22a33e 100644 --- a/rust/src/operations/vacuum.rs +++ b/rust/src/operations/vacuum.rs @@ -33,7 +33,7 @@ use object_store::{path::Path, ObjectStore}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::DeltaTable; /// Errors that can occur during vacuum diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index ca96134935..861e9a9dff 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -32,15 +32,15 @@ use serde_json::Map; use super::writer::{DeltaWriter, WriterConfig}; use super::MAX_SUPPORTED_WRITER_VERSION; use super::{transaction::commit, CreateBuilder}; -use crate::action::{Action, Add, DeltaOperation, Remove, SaveMode}; -use crate::delta::DeltaTable; use crate::delta_datafusion::DeltaDataChecker; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::{Action, Add, DeltaOperation, Remove, SaveMode}; use crate::schema::Schema; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; -use crate::table_state::DeltaTableState; +use crate::table::state::DeltaTableState; use crate::writer::record_batch::divide_by_partition_values; use crate::writer::utils::PartitionPath; +use crate::DeltaTable; #[derive(thiserror::Error, Debug)] enum WriteError { @@ -552,8 +552,8 @@ fn cast_record_batch( #[cfg(test)] mod tests { use super::*; - use crate::action::SaveMode; use crate::operations::{collect_sendable_stream, DeltaOps}; + use crate::protocol::SaveMode; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::{ get_delta_schema, get_delta_schema_with_nested_struct, get_record_batch, diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index a72b832505..05bda44ae6 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -2,16 +2,6 @@ use std::collections::HashMap; -use crate::action::Add; -use crate::storage::ObjectStoreRef; -use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; -use crate::writer::stats::create_add; -use crate::writer::utils::{ - self, arrow_schema_without_partitions, record_batch_without_partitions, PartitionPath, - ShareableBuffer, -}; -use crate::{crate_version, DeltaResult, DeltaTableError}; - use arrow::datatypes::SchemaRef as ArrowSchemaRef; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; @@ -21,6 +11,17 @@ use parquet::arrow::ArrowWriter; use parquet::basic::Compression; use parquet::file::properties::WriterProperties; +use crate::crate_version; +use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; +use crate::storage::ObjectStoreRef; +use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; +use crate::writer::stats::create_add; +use crate::writer::utils::{ + arrow_schema_without_partitions, next_data_path, record_batch_without_partitions, + PartitionPath, ShareableBuffer, +}; + // TODO databricks often suggests a file size of 100mb, should we set this default? const DEFAULT_TARGET_FILE_SIZE: usize = 104_857_600; const DEFAULT_WRITE_BATCH_SIZE: usize = 1024; @@ -304,7 +305,7 @@ impl PartitionWriter { fn next_data_path(&mut self) -> Path { self.part_counter += 1; - utils::next_data_path( + next_data_path( &self.config.prefix, self.part_counter, &self.writer_id, diff --git a/rust/src/action/checkpoints.rs b/rust/src/protocol/checkpoints.rs similarity index 98% rename from rust/src/action/checkpoints.rs rename to rust/src/protocol/checkpoints.rs index 86dbc7bc87..3bf2eb962e 100644 --- a/rust/src/action/checkpoints.rs +++ b/rust/src/protocol/checkpoints.rs @@ -19,12 +19,13 @@ use parquet::errors::ParquetError; use regex::Regex; use serde_json::Value; -use super::{Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; -use crate::delta_arrow::delta_log_schema_for_table; +use super::{time_utils, Action, Add as AddAction, MetaData, Protocol, ProtocolError, Txn}; +use crate::arrow_convert::delta_log_schema_for_table; +use crate::schema::*; use crate::storage::DeltaObjectStore; -use crate::table_state::DeltaTableState; -use crate::{open_table_with_version, time_utils, CheckPoint, DeltaTable}; -use crate::{schema::*, CheckPointBuilder}; +use crate::table::state::DeltaTableState; +use crate::table::{CheckPoint, CheckPointBuilder}; +use crate::{open_table_with_version, DeltaTable}; type SchemaPath = Vec; diff --git a/rust/src/action/mod.rs b/rust/src/protocol/mod.rs similarity index 99% rename from rust/src/action/mod.rs rename to rust/src/protocol/mod.rs index 105ebdcdab..b03ebe4a9a 100644 --- a/rust/src/action/mod.rs +++ b/rust/src/protocol/mod.rs @@ -9,6 +9,7 @@ pub mod parquet2_read; #[cfg(feature = "parquet")] mod parquet_read; mod serde_path; +mod time_utils; #[cfg(feature = "arrow")] use arrow_schema::ArrowError; @@ -25,10 +26,11 @@ use std::hash::{Hash, Hasher}; use std::mem::take; use std::str::FromStr; -use crate::delta_config::IsolationLevel; use crate::errors::DeltaResult; use crate::storage::ObjectStoreRef; -use crate::{delta::CheckPoint, schema::*, DeltaTableMetaData}; +use crate::table::config::IsolationLevel; +use crate::table::DeltaTableMetaData; +use crate::{schema::*, table::CheckPoint}; /// Error returned when an invalid Delta log action is encountered. #[allow(missing_docs)] diff --git a/rust/src/action/parquet2_read/boolean.rs b/rust/src/protocol/parquet2_read/boolean.rs similarity index 98% rename from rust/src/action/parquet2_read/boolean.rs rename to rust/src/protocol/parquet2_read/boolean.rs index 553ba36709..474a61a153 100644 --- a/rust/src/action/parquet2_read/boolean.rs +++ b/rust/src/protocol/parquet2_read/boolean.rs @@ -4,7 +4,7 @@ use parquet2::page::DataPage; use super::validity::ValidityRowIndexIter; use super::{split_page, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; /// Parquet dictionary primitive value reader pub struct SomeBooleanValueIter<'a> { diff --git a/rust/src/action/parquet2_read/dictionary/binary.rs b/rust/src/protocol/parquet2_read/dictionary/binary.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/binary.rs rename to rust/src/protocol/parquet2_read/dictionary/binary.rs diff --git a/rust/src/action/parquet2_read/dictionary/mod.rs b/rust/src/protocol/parquet2_read/dictionary/mod.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/mod.rs rename to rust/src/protocol/parquet2_read/dictionary/mod.rs diff --git a/rust/src/action/parquet2_read/dictionary/primitive.rs b/rust/src/protocol/parquet2_read/dictionary/primitive.rs similarity index 100% rename from rust/src/action/parquet2_read/dictionary/primitive.rs rename to rust/src/protocol/parquet2_read/dictionary/primitive.rs diff --git a/rust/src/action/parquet2_read/map.rs b/rust/src/protocol/parquet2_read/map.rs similarity index 99% rename from rust/src/action/parquet2_read/map.rs rename to rust/src/protocol/parquet2_read/map.rs index ed730e383b..0739feae2d 100644 --- a/rust/src/action/parquet2_read/map.rs +++ b/rust/src/protocol/parquet2_read/map.rs @@ -3,7 +3,7 @@ use parquet2::page::{DataPage, DictPage}; use super::string::for_each_repeated_string_field_value_with_idx; use super::{ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; #[derive(Default)] pub struct MapState { diff --git a/rust/src/action/parquet2_read/mod.rs b/rust/src/protocol/parquet2_read/mod.rs similarity index 99% rename from rust/src/action/parquet2_read/mod.rs rename to rust/src/protocol/parquet2_read/mod.rs index afa6065279..28908fe6bd 100644 --- a/rust/src/action/parquet2_read/mod.rs +++ b/rust/src/protocol/parquet2_read/mod.rs @@ -11,7 +11,7 @@ use parquet2::read::get_page_iterator; use parquet2::read::levels::get_bit_width; use super::ProtocolError; -use crate::action::{Action, Add, CommitInfo, MetaData, Protocol, Remove, Txn}; +use crate::protocol::{Action, Add, CommitInfo, MetaData, Protocol, Remove, Txn}; use crate::schema::Guid; use boolean::for_each_boolean_field_value; use map::for_each_map_field_value; @@ -751,7 +751,7 @@ mod tests { assert_eq!(meta_data.description, None); assert_eq!( meta_data.format, - crate::action::Format::new("parquet".to_string(), None), + crate::protocol::Format::new("parquet".to_string(), None), ); assert_eq!(meta_data.schema_string, "{\"type\":\"struct\",\"fields\":[{\"name\":\"value\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"); assert_eq!(meta_data.partition_columns.len(), 0); @@ -826,7 +826,7 @@ mod tests { assert_eq!(meta_data.description, None); assert_eq!( meta_data.format, - crate::action::Format::new("parquet".to_string(), None), + crate::protocol::Format::new("parquet".to_string(), None), ); assert_eq!( meta_data.schema_string, diff --git a/rust/src/action/parquet2_read/primitive.rs b/rust/src/protocol/parquet2_read/primitive.rs similarity index 99% rename from rust/src/action/parquet2_read/primitive.rs rename to rust/src/protocol/parquet2_read/primitive.rs index 4f262f7aaa..29147ea8ca 100644 --- a/rust/src/action/parquet2_read/primitive.rs +++ b/rust/src/protocol/parquet2_read/primitive.rs @@ -12,7 +12,7 @@ use parquet2::types::NativeType; use super::dictionary; use super::validity::ValidityRowIndexIter; use super::{split_page, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; struct ExactChunksIter<'a, T: NativeType> { chunks: std::slice::ChunksExact<'a, u8>, diff --git a/rust/src/action/parquet2_read/stats.rs b/rust/src/protocol/parquet2_read/stats.rs similarity index 85% rename from rust/src/action/parquet2_read/stats.rs rename to rust/src/protocol/parquet2_read/stats.rs index c9bb2f9bdc..689dfea6c0 100644 --- a/rust/src/action/parquet2_read/stats.rs +++ b/rust/src/protocol/parquet2_read/stats.rs @@ -1,4 +1,4 @@ -use crate::action::{Add, ProtocolError, Stats}; +use crate::protocol::{Add, ProtocolError, Stats}; impl Add { /// Returns the composite HashMap representation of stats contained in the action if present. diff --git a/rust/src/action/parquet2_read/string.rs b/rust/src/protocol/parquet2_read/string.rs similarity index 99% rename from rust/src/action/parquet2_read/string.rs rename to rust/src/protocol/parquet2_read/string.rs index 1a851aec3b..fc0ec574e0 100644 --- a/rust/src/action/parquet2_read/string.rs +++ b/rust/src/protocol/parquet2_read/string.rs @@ -9,7 +9,7 @@ use super::dictionary; use super::dictionary::binary::BinaryPageDict; use super::validity::{ValidityRepeatedRowIndexIter, ValidityRowIndexIter}; use super::{split_page, split_page_nested, ActionVariant, ParseError}; -use crate::action::Action; +use crate::protocol::Action; pub trait StringValueIter<'a>: Iterator> { fn try_from_encoded_values( diff --git a/rust/src/action/parquet2_read/validity.rs b/rust/src/protocol/parquet2_read/validity.rs similarity index 100% rename from rust/src/action/parquet2_read/validity.rs rename to rust/src/protocol/parquet2_read/validity.rs diff --git a/rust/src/action/parquet_read/mod.rs b/rust/src/protocol/parquet_read/mod.rs similarity index 99% rename from rust/src/action/parquet_read/mod.rs rename to rust/src/protocol/parquet_read/mod.rs index 3ff6dfa710..93fdc4c2df 100644 --- a/rust/src/action/parquet_read/mod.rs +++ b/rust/src/protocol/parquet_read/mod.rs @@ -6,7 +6,7 @@ use num_traits::cast::ToPrimitive; use parquet::record::{Field, ListAccessor, MapAccessor, RowAccessor}; use serde_json::json; -use crate::action::{ +use crate::protocol::{ Action, Add, AddCDCFile, ColumnCountStat, ColumnValueStat, DeletionVector, MetaData, Protocol, ProtocolError, Remove, Stats, Txn, }; diff --git a/rust/src/action/serde_path.rs b/rust/src/protocol/serde_path.rs similarity index 100% rename from rust/src/action/serde_path.rs rename to rust/src/protocol/serde_path.rs diff --git a/rust/src/time_utils.rs b/rust/src/protocol/time_utils.rs similarity index 100% rename from rust/src/time_utils.rs rename to rust/src/protocol/time_utils.rs diff --git a/rust/src/delta_arrow.rs b/rust/src/schema/arrow_convert.rs similarity index 100% rename from rust/src/delta_arrow.rs rename to rust/src/schema/arrow_convert.rs diff --git a/rust/src/schema.rs b/rust/src/schema/mod.rs similarity index 99% rename from rust/src/schema.rs rename to rust/src/schema/mod.rs index 2602c5cd68..a853725fc6 100644 --- a/rust/src/schema.rs +++ b/rust/src/schema/mod.rs @@ -8,6 +8,10 @@ use std::collections::HashMap; use crate::errors::DeltaTableError; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod arrow_convert; +pub mod partitions; + /// Type alias for a string expected to match a GUID/UUID format pub type Guid = String; diff --git a/rust/src/partitions.rs b/rust/src/schema/partitions.rs similarity index 90% rename from rust/src/partitions.rs rename to rust/src/schema/partitions.rs index ed4a5a2eaf..0c1b0f6404 100644 --- a/rust/src/partitions.rs +++ b/rust/src/schema/partitions.rs @@ -2,7 +2,7 @@ use std::convert::TryFrom; -use super::schema::SchemaDataType; +use super::SchemaDataType; use crate::errors::DeltaTableError; use std::cmp::Ordering; use std::collections::HashMap; @@ -189,20 +189,18 @@ pub struct DeltaTablePartition<'a> { pub value: &'a str, } -/** - * Create a DeltaTable partition from a HivePartition string. - * - * A HivePartition string is represented by a "key=value" format. - * - * ```rust - * use deltalake::DeltaTablePartition; - * - * let hive_part = "ds=2023-01-01"; - * let partition = DeltaTablePartition::try_from(hive_part).unwrap(); - * assert_eq!("ds", partition.key); - * assert_eq!("2023-01-01", partition.value); - * ``` - */ +/// Create a DeltaTable partition from a HivePartition string. +/// +/// A HivePartition string is represented by a "key=value" format. +/// +/// ```rust +/// use deltalake::DeltaTablePartition; +/// +/// let hive_part = "ds=2023-01-01"; +/// let partition = DeltaTablePartition::try_from(hive_part).unwrap(); +/// assert_eq!("ds", partition.key); +/// assert_eq!("2023-01-01", partition.value); +/// ``` impl<'a> TryFrom<&'a str> for DeltaTablePartition<'a> { type Error = DeltaTableError; @@ -223,20 +221,18 @@ impl<'a> TryFrom<&'a str> for DeltaTablePartition<'a> { } impl<'a> DeltaTablePartition<'a> { - /** - * Try to create a DeltaTable partition from a partition value kv pair. - * - * ```rust - * use deltalake::DeltaTablePartition; - * - * let value = (&"ds".to_string(), &Some("2023-01-01".to_string())); - * let null_default = "1979-01-01"; - * let partition = DeltaTablePartition::from_partition_value(value, null_default); - * - * assert_eq!("ds", partition.key); - * assert_eq!("2023-01-01", partition.value); - * ``` - */ + /// Try to create a DeltaTable partition from a partition value kv pair. + /// + /// ```rust + /// use deltalake::DeltaTablePartition; + /// + /// let value = (&"ds".to_string(), &Some("2023-01-01".to_string())); + /// let null_default = "1979-01-01"; + /// let partition = DeltaTablePartition::from_partition_value(value, null_default); + /// + /// assert_eq!("ds", partition.key); + /// assert_eq!("2023-01-01", partition.value); + /// ``` pub fn from_partition_value( partition_value: (&'a String, &'a Option), default_for_null: &'a str, diff --git a/rust/src/storage/config.rs b/rust/src/storage/config.rs index 76c2e6dd81..1cba57b579 100644 --- a/rust/src/storage/config.rs +++ b/rust/src/storage/config.rs @@ -299,7 +299,7 @@ fn url_prefix_handler(store: T, prefix: Path) -> DeltaResult + Clone) -> DeltaResult { let mut options = options.into(); let storage = config::configure_store(&location, &mut options)?; diff --git a/rust/src/storage/s3.rs b/rust/src/storage/s3.rs index ec5e6a344a..2cc33f3ae3 100644 --- a/rust/src/storage/s3.rs +++ b/rust/src/storage/s3.rs @@ -1,7 +1,7 @@ //! AWS S3 storage backend. use super::utils::str_is_truthy; -use crate::builder::{s3_storage_options, str_option}; +use crate::table::builder::{s3_storage_options, str_option}; use bytes::Bytes; use dynamodb_lock::{DynamoError, LockClient, LockItem, DEFAULT_MAX_RETRY_ACQUIRE_LOCK_ATTEMPTS}; use futures::stream::BoxStream; diff --git a/rust/src/storage/utils.rs b/rust/src/storage/utils.rs index 7cb27d721a..80710efd9b 100644 --- a/rust/src/storage/utils.rs +++ b/rust/src/storage/utils.rs @@ -8,9 +8,9 @@ use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; -use crate::action::Add; -use crate::builder::DeltaTableBuilder; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::protocol::Add; +use crate::table::builder::DeltaTableBuilder; /// Copies the contents from the `from` location into the `to` location pub async fn copy_table( @@ -82,7 +82,7 @@ impl TryFrom<&Add> for ObjectMeta { fn try_from(value: &Add) -> DeltaResult { let last_modified = Utc.from_utc_datetime( &NaiveDateTime::from_timestamp_millis(value.modification_time).ok_or( - DeltaTableError::from(crate::action::ProtocolError::InvalidField(format!( + DeltaTableError::from(crate::protocol::ProtocolError::InvalidField(format!( "invalid modification_time: {:?}", value.modification_time ))), diff --git a/rust/src/builder.rs b/rust/src/table/builder.rs similarity index 99% rename from rust/src/builder.rs rename to rust/src/table/builder.rs index decfda8db0..92fc4851ad 100644 --- a/rust/src/builder.rs +++ b/rust/src/table/builder.rs @@ -9,7 +9,7 @@ use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; use url::Url; -use crate::delta::DeltaTable; +use super::DeltaTable; use crate::errors::{DeltaResult, DeltaTableError}; use crate::storage::config::StorageOptions; use crate::storage::{DeltaObjectStore, ObjectStoreRef}; diff --git a/rust/src/delta_config.rs b/rust/src/table/config.rs similarity index 99% rename from rust/src/delta_config.rs rename to rust/src/table/config.rs index 1b7f6b7e0f..60498767ab 100644 --- a/rust/src/delta_config.rs +++ b/rust/src/table/config.rs @@ -387,7 +387,7 @@ fn parse_int(value: &str) -> Result { #[cfg(test)] mod tests { use super::*; - use crate::DeltaTableMetaData; + use crate::table::DeltaTableMetaData; use crate::Schema; use std::collections::HashMap; diff --git a/rust/src/delta.rs b/rust/src/table/mod.rs similarity index 92% rename from rust/src/delta.rs rename to rust/src/table/mod.rs index d877b77191..667b2239aa 100644 --- a/rust/src/delta.rs +++ b/rust/src/table/mod.rs @@ -1,7 +1,5 @@ //! Delta Table read and write implementation -// Reference: https://github.com/delta-io/delta/blob/master/PROTOCOL.md -// use std::collections::HashMap; use std::convert::TryFrom; use std::fmt; @@ -21,17 +19,20 @@ use serde::ser::SerializeSeq; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use uuid::Uuid; -use super::action; -use super::action::{find_latest_check_point_for_version, get_last_checkpoint, Action}; -use super::partitions::PartitionFilter; -use super::schema::*; -use super::table_state::DeltaTableState; -use crate::action::{Add, ProtocolError, Stats}; +use self::builder::DeltaTableConfig; +use self::state::DeltaTableState; use crate::errors::DeltaTableError; +use crate::partitions::PartitionFilter; +use crate::protocol::{self, find_latest_check_point_for_version, get_last_checkpoint, Action}; +use crate::protocol::{Add, ProtocolError, Stats}; +use crate::schema::*; use crate::storage::{commit_uri_from_version, ObjectStoreRef}; -// TODO re-exports only for transition -pub use crate::builder::{DeltaTableBuilder, DeltaTableConfig, DeltaVersion}; +pub mod builder; +pub mod config; +pub mod state; +#[cfg(feature = "arrow")] +pub mod state_arrow; /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] @@ -136,7 +137,7 @@ pub struct DeltaTableMetaData { /// User-provided description for this table pub description: Option, /// Specification of the encoding for the files stored in the table - pub format: action::Format, + pub format: protocol::Format, /// Schema of the table pub schema: Schema, /// An array containing the names of columns by which the data should be partitioned @@ -152,7 +153,7 @@ impl DeltaTableMetaData { pub fn new( name: Option, description: Option, - format: Option, + format: Option, schema: Schema, partition_columns: Vec, configuration: HashMap>, @@ -208,10 +209,10 @@ impl fmt::Display for DeltaTableMetaData { } } -impl TryFrom for DeltaTableMetaData { +impl TryFrom for DeltaTableMetaData { type Error = ProtocolError; - fn try_from(action_metadata: action::MetaData) -> Result { + fn try_from(action_metadata: protocol::MetaData) -> Result { let schema = action_metadata.get_schema()?; Ok(Self { id: action_metadata.id, @@ -663,7 +664,7 @@ impl DeltaTable { pub async fn history( &mut self, limit: Option, - ) -> Result, DeltaTableError> { + ) -> Result, DeltaTableError> { let mut version = match limit { Some(l) => max(self.version() - l as i64 + 1, 0), None => self.get_earliest_delta_log_version().await?, @@ -796,7 +797,7 @@ impl DeltaTable { } /// Returns a vector of active tombstones (i.e. `Remove` actions present in the current delta log). - pub fn get_tombstones(&self) -> impl Iterator { + pub fn get_tombstones(&self) -> impl Iterator { self.state.unexpired_tombstones() } @@ -907,64 +908,12 @@ impl std::fmt::Debug for DeltaTable { } } -/// Creates and loads a DeltaTable from the given path with current metadata. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table(table_uri: impl AsRef) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri).load().await?; - Ok(table) -} - -/// Same as `open_table`, but also accepts storage options to aid in building the table for a deduced -/// `StorageService`. -pub async fn open_table_with_storage_options( - table_uri: impl AsRef, - storage_options: HashMap, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_storage_options(storage_options) - .load() - .await?; - Ok(table) -} - -/// Creates a DeltaTable from the given path and loads it with the metadata from the given version. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table_with_version( - table_uri: impl AsRef, - version: i64, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_version(version) - .load() - .await?; - Ok(table) -} - -/// Creates a DeltaTable from the given path. -/// Loads metadata from the version appropriate based on the given ISO-8601/RFC-3339 timestamp. -/// Infers the storage backend to use from the scheme in the given table path. -pub async fn open_table_with_ds( - table_uri: impl AsRef, - ds: impl AsRef, -) -> Result { - let table = DeltaTableBuilder::from_uri(table_uri) - .with_datestring(ds)? - .load() - .await?; - Ok(table) -} - -/// Returns rust crate version, can be use used in language bindings to expose Rust core version -pub fn crate_version() -> &'static str { - env!("CARGO_PKG_VERSION") -} - #[cfg(test)] mod tests { use super::*; - #[cfg(any(feature = "s3", feature = "s3-native-tls"))] - use crate::builder::DeltaTableBuilder; use crate::operations::create::CreateBuilder; + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] + use crate::table::builder::DeltaTableBuilder; use pretty_assertions::assert_eq; use std::collections::HashMap; use tempdir::TempDir; diff --git a/rust/src/table_state.rs b/rust/src/table/state.rs similarity index 91% rename from rust/src/table_state.rs rename to rust/src/table/state.rs index 2ac17032d4..71aa6bddc9 100644 --- a/rust/src/table_state.rs +++ b/rust/src/table/state.rs @@ -10,14 +10,15 @@ use lazy_static::lazy_static; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use serde::{Deserialize, Serialize}; -use crate::action::{self, Action, Add, ProtocolError}; -use crate::delta_config::TableConfig; +use super::config::TableConfig; use crate::errors::DeltaTableError; use crate::partitions::{DeltaTablePartition, PartitionFilter}; +use crate::protocol::{self, Action, Add, ProtocolError}; use crate::schema::SchemaDataType; use crate::storage::commit_uri_from_version; +use crate::table::DeltaTableMetaData; +use crate::DeltaTable; use crate::Schema; -use crate::{DeltaTable, DeltaTableMetaData}; #[cfg(any(feature = "parquet", feature = "parquet2"))] use super::{CheckPoint, DeltaTableConfig}; @@ -30,13 +31,13 @@ pub struct DeltaTableState { version: i64, // A remove action should remain in the state of the table as a tombstone until it has expired. // A tombstone expires when the creation timestamp of the delta file exceeds the expiration - tombstones: HashSet, + tombstones: HashSet, // active files for table state - files: Vec, + files: Vec, // Information added to individual commits - commit_infos: Vec, + commit_infos: Vec, // Domain metadatas provided by the system or user - domain_metadatas: Vec, + domain_metadatas: Vec, app_transaction_version: HashMap, min_reader_version: i32, min_writer_version: i32, @@ -75,7 +76,7 @@ impl DeltaTableState { let mut new_state = DeltaTableState::with_version(version); for line in reader.lines() { - let action: action::Action = serde_json::from_str(line?.as_str())?; + let action: protocol::Action = serde_json::from_str(line?.as_str())?; new_state.process_action( action, table.config.require_tombstones, @@ -109,13 +110,13 @@ impl DeltaTableState { let preader = SerializedFileReader::new(data)?; let schema = preader.metadata().file_metadata().schema(); if !schema.is_group() { - return Err(DeltaTableError::from(action::ProtocolError::Generic( + return Err(DeltaTableError::from(protocol::ProtocolError::Generic( "Action record in checkpoint should be a struct".to_string(), ))); } for record in preader.get_row_iter(None)? { self.process_action( - action::Action::from_parquet_record(schema, &record.unwrap())?, + protocol::Action::from_parquet_record(schema, &record.unwrap())?, table_config.require_tombstones, table_config.require_files, )?; @@ -124,7 +125,7 @@ impl DeltaTableState { #[cfg(feature = "parquet2")] { - use crate::action::parquet2_read::actions_from_row_group; + use crate::protocol::parquet2_read::actions_from_row_group; use parquet2::read::read_metadata; let mut reader = std::io::Cursor::new(data); @@ -132,7 +133,7 @@ impl DeltaTableState { for row_group in metadata.row_groups { for action in actions_from_row_group(row_group, &mut reader) - .map_err(action::ProtocolError::from)? + .map_err(protocol::ProtocolError::from)? { self.process_action( action, @@ -164,7 +165,7 @@ impl DeltaTableState { } /// List of commit info maps. - pub fn commit_infos(&self) -> &Vec { + pub fn commit_infos(&self) -> &Vec { &self.commit_infos } @@ -184,13 +185,13 @@ impl DeltaTableState { } /// Full list of tombstones (remove actions) representing files removed from table state). - pub fn all_tombstones(&self) -> &HashSet { + pub fn all_tombstones(&self) -> &HashSet { &self.tombstones } /// List of unexpired tombstones (remove actions) representing files removed from table state. /// The retention period is set by `deletedFileRetentionDuration` with default value of 1 week. - pub fn unexpired_tombstones(&self) -> impl Iterator { + pub fn unexpired_tombstones(&self) -> impl Iterator { let retention_timestamp = Utc::now().timestamp_millis() - self.tombstone_retention_millis; self.tombstones .iter() @@ -199,7 +200,7 @@ impl DeltaTableState { /// Full list of add actions representing all parquet files that are part of the current /// delta table state. - pub fn files(&self) -> &Vec { + pub fn files(&self) -> &Vec { self.files.as_ref() } @@ -318,28 +319,28 @@ impl DeltaTableState { /// Process given action by updating current state. fn process_action( &mut self, - action: action::Action, + action: protocol::Action, require_tombstones: bool, require_files: bool, ) -> Result<(), ProtocolError> { match action { // TODO: optionally load CDC into TableState - action::Action::cdc(_v) => {} - action::Action::add(v) => { + protocol::Action::cdc(_v) => {} + protocol::Action::add(v) => { if require_files { self.files.push(v); } } - action::Action::remove(v) => { + protocol::Action::remove(v) => { if require_tombstones && require_files { self.tombstones.insert(v); } } - action::Action::protocol(v) => { + protocol::Action::protocol(v) => { self.min_reader_version = v.min_reader_version; self.min_writer_version = v.min_writer_version; } - action::Action::metaData(v) => { + protocol::Action::metaData(v) => { let md = DeltaTableMetaData::try_from(v)?; let table_config = TableConfig(&md.configuration); self.tombstone_retention_millis = @@ -349,16 +350,16 @@ impl DeltaTableState { self.enable_expired_log_cleanup = table_config.enable_expired_log_cleanup(); self.current_metadata = Some(md); } - action::Action::txn(v) => { + protocol::Action::txn(v) => { *self .app_transaction_version .entry(v.app_id) .or_insert(v.version) = v.version; } - action::Action::commitInfo(v) => { + protocol::Action::commitInfo(v) => { self.commit_infos.push(v); } - action::Action::domainMetadata(v) => { + protocol::Action::domainMetadata(v) => { self.domain_metadatas.push(v); } } @@ -451,7 +452,7 @@ mod tests { enable_expired_log_cleanup: true, }; - let txn_action = action::Action::txn(action::Txn { + let txn_action = protocol::Action::txn(protocol::Txn { app_id: "abc".to_string(), version: 2, last_updated: Some(0), diff --git a/rust/src/table_state_arrow.rs b/rust/src/table/state_arrow.rs similarity index 99% rename from rust/src/table_state_arrow.rs rename to rust/src/table/state_arrow.rs index d4765d48d8..34f858f415 100644 --- a/rust/src/table_state_arrow.rs +++ b/rust/src/table/state_arrow.rs @@ -1,6 +1,6 @@ //! Methods to get Delta Table state in Arrow structures //! -//! See [crate::table_state::DeltaTableState]. +//! See [crate::table::DeltaTableState]. use std::borrow::Cow; use std::collections::{HashMap, HashSet, VecDeque}; @@ -16,9 +16,9 @@ use arrow_array::{ use arrow_schema::{DataType, Field, Fields, TimeUnit}; use itertools::Itertools; -use crate::action::{ColumnCountStat, ColumnValueStat, Stats}; +use super::state::DeltaTableState; use crate::errors::DeltaTableError; -use crate::table_state::DeltaTableState; +use crate::protocol::{ColumnCountStat, ColumnValueStat, Stats}; use crate::SchemaDataType; use crate::SchemaTypeStruct; diff --git a/rust/src/test_utils.rs b/rust/src/test_utils.rs index 352a46d2b3..124ec0365b 100644 --- a/rust/src/test_utils.rs +++ b/rust/src/test_utils.rs @@ -385,7 +385,7 @@ pub mod az_cli { /// small wrapper around s3 cli pub mod s3_cli { use super::set_env_if_not_set; - use crate::builder::s3_storage_options; + use crate::table::builder::s3_storage_options; use std::process::{Command, ExitStatus, Stdio}; /// Create a new bucket diff --git a/rust/src/writer/json.rs b/rust/src/writer/json.rs index fc98cb0b90..f8d6d1a9e3 100644 --- a/rust/src/writer/json.rs +++ b/rust/src/writer/json.rs @@ -3,16 +3,6 @@ use std::collections::HashMap; use std::convert::TryFrom; use std::sync::Arc; -use super::stats::create_add; -use super::utils::{ - arrow_schema_without_partitions, next_data_path, record_batch_from_message, - record_batch_without_partitions, stringified_partition_value, PartitionPath, -}; -use super::{DeltaWriter, DeltaWriterError}; -use crate::builder::DeltaTableBuilder; -use crate::{action::Add, DeltaTable, DeltaTableError, DeltaTableMetaData, Schema}; -use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; - use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use arrow::record_batch::*; use bytes::Bytes; @@ -26,6 +16,18 @@ use parquet::{ use serde_json::Value; use uuid::Uuid; +use super::stats::create_add; +use super::utils::{ + arrow_schema_without_partitions, next_data_path, record_batch_from_message, + record_batch_without_partitions, stringified_partition_value, +}; +use super::{utils::PartitionPath, DeltaWriter, DeltaWriterError}; +use crate::errors::DeltaTableError; +use crate::table::builder::DeltaTableBuilder; +use crate::table::DeltaTableMetaData; +use crate::{protocol::Add, DeltaTable, Schema}; +use crate::{storage::DeltaObjectStore, writer::utils::ShareableBuffer}; + type BadValue = (Value, ParquetError); /// Writes messages to a delta lake table. diff --git a/rust/src/writer/mod.rs b/rust/src/writer/mod.rs index 5685a71d48..8c5512127f 100644 --- a/rust/src/writer/mod.rs +++ b/rust/src/writer/mod.rs @@ -7,9 +7,9 @@ use object_store::Error as ObjectStoreError; use parquet::errors::ParquetError; use serde_json::Value; -use crate::action::{Action, Add, ColumnCountStat, DeltaOperation, SaveMode}; use crate::errors::DeltaTableError; use crate::operations::transaction::commit; +use crate::protocol::{Action, Add, ColumnCountStat, DeltaOperation, SaveMode}; use crate::DeltaTable; pub use json::JsonWriter; diff --git a/rust/src/writer/record_batch.rs b/rust/src/writer/record_batch.rs index 986b7b4026..e6495b6539 100644 --- a/rust/src/writer/record_batch.rs +++ b/rust/src/writer/record_batch.rs @@ -4,37 +4,15 @@ //! Each Parquet file is buffered in-memory and only written once `flush()` is called on //! the writer. Once written, add actions are returned by the writer. It's the users responsibility //! to create the transaction using those actions. -//! -//! # Examples -//! -//! Write to an existing Delta Lake table: -//! ```rust ignore -//! let table = DeltaTable::try_from_uri("../path/to/table") -//! let batch: RecordBatch = ... -//! let mut writer = RecordBatchWriter::for_table(table, /*storage_options=*/ HashMap::new()) -//! writer.write(batch)?; -//! let actions: Vec = writer.flush()?.iter() -//! .map(|add| Action::add(add.into())) -//! .collect(); -//! let mut transaction = table.create_transaction(Some(DeltaTransactionOptions::new(/*max_retry_attempts=*/3))); -//! transaction.add_actions(actions); -//! async { -//! transaction.commit(Some(DeltaOperation::Write { -//! SaveMode::Append, -//! partitionBy: Some(table.get_metadata().partition_columns), -//! predicate: None, -//! })) -//! } -//! ``` + use std::{collections::HashMap, sync::Arc}; use arrow::array::{Array, UInt32Array}; use arrow::compute::{partition, take}; -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; -use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use arrow_array::ArrayRef; use arrow_row::{RowConverter, SortField}; +use arrow_schema::{ArrowError, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; use object_store::{path::Path, ObjectStore}; use parquet::{arrow::ArrowWriter, errors::ParquetError}; @@ -47,9 +25,10 @@ use super::utils::{ stringified_partition_value, PartitionPath, ShareableBuffer, }; use super::{DeltaWriter, DeltaWriterError}; -use crate::builder::DeltaTableBuilder; use crate::errors::DeltaTableError; -use crate::{action::Add, storage::DeltaObjectStore, DeltaTable, DeltaTableMetaData, Schema}; +use crate::table::builder::DeltaTableBuilder; +use crate::table::DeltaTableMetaData; +use crate::{protocol::Add, storage::DeltaObjectStore, DeltaTable, Schema}; /// Writes messages to a delta lake table. pub struct RecordBatchWriter { diff --git a/rust/src/writer/stats.rs b/rust/src/writer/stats.rs index d96508d14f..0d369de46d 100644 --- a/rust/src/writer/stats.rs +++ b/rust/src/writer/stats.rs @@ -11,7 +11,7 @@ use parquet::{ }; use super::*; -use crate::action::{Add, ColumnValueStat, Stats}; +use crate::protocol::{Add, ColumnValueStat, Stats}; /// Creates an [`Add`] log action struct. pub fn create_add( @@ -479,9 +479,9 @@ mod tests { use super::utils::record_batch_from_message; use super::*; use crate::{ - action::{ColumnCountStat, ColumnValueStat}, - builder::DeltaTableBuilder, errors::DeltaTableError, + protocol::{ColumnCountStat, ColumnValueStat}, + table::builder::DeltaTableBuilder, DeltaTable, }; use lazy_static::lazy_static; diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index f519ebf720..d6c79bee94 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -1,3 +1,5 @@ +//! Utilities for writing unit tests + use std::collections::HashMap; use std::sync::Arc; @@ -5,10 +7,10 @@ use arrow::compute::take; use arrow_array::{Int32Array, Int64Array, RecordBatch, StringArray, StructArray, UInt32Array}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; -use crate::delta::DeltaTableMetaData; use crate::operations::create::CreateBuilder; -use crate::schema::Schema; -use crate::{DeltaTable, DeltaTableBuilder, SchemaDataType, SchemaField, SchemaTypeStruct}; +use crate::schema::{Schema, SchemaTypeStruct}; +use crate::table::DeltaTableMetaData; +use crate::{DeltaTable, DeltaTableBuilder, SchemaDataType, SchemaField}; pub type TestResult = Result<(), Box>; diff --git a/rust/tests/checkpoint_writer.rs b/rust/tests/checkpoint_writer.rs index 999f07ff9c..10af3d9cfa 100644 --- a/rust/tests/checkpoint_writer.rs +++ b/rust/tests/checkpoint_writer.rs @@ -1,7 +1,6 @@ -#![deny(warnings)] - #[cfg(all(feature = "arrow", feature = "parquet"))] mod fs_common; +use deltalake::protocol::DeltaOperation; // NOTE: The below is a useful external command for inspecting the written checkpoint schema visually: // parquet-tools inspect tests/data/checkpoints/_delta_log/00000000000000000005.checkpoint.parquet @@ -91,7 +90,7 @@ mod delete_expired_delta_log_in_checkpoint { use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; - use deltalake::delta_config::DeltaConfigKey; + use deltalake::table::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; @@ -212,8 +211,8 @@ mod checkpoints_with_tombstones { use super::*; use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; - use deltalake::action::*; - use deltalake::delta_config::DeltaConfigKey; + use deltalake::protocol::*; + use deltalake::table::config::DeltaConfigKey; use deltalake::*; use maplit::hashmap; use parquet::file::reader::{FileReader, SerializedFileReader}; @@ -361,7 +360,6 @@ mod checkpoints_with_tombstones { .map(Action::remove) .chain(std::iter::once(Action::add(add.clone()))) .collect(); - let operation = DeltaOperation::Optimize { predicate: None, target_size: 1000000, diff --git a/rust/tests/command_filesystem_check.rs b/rust/tests/command_filesystem_check.rs index ced317d990..86fcd8f52e 100644 --- a/rust/tests/command_filesystem_check.rs +++ b/rust/tests/command_filesystem_check.rs @@ -12,7 +12,7 @@ mod common; #[tokio::test] #[serial] async fn test_filesystem_check_local() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Local).await?) + test_filesystem_check(StorageIntegration::Local).await } #[cfg(any(feature = "s3", feature = "s3-native-tls"))] @@ -21,21 +21,21 @@ async fn test_filesystem_check_local() -> TestResult { async fn test_filesystem_check_aws() -> TestResult { set_env_if_not_set("AWS_S3_ALLOW_UNSAFE_RENAME", "true"); set_env_if_not_set("AWS_S3_LOCKING_PROVIDER", "none"); - Ok(test_filesystem_check(StorageIntegration::Amazon).await?) + test_filesystem_check(StorageIntegration::Amazon).await } #[cfg(feature = "azure")] #[tokio::test] #[serial] async fn test_filesystem_check_azure() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Microsoft).await?) + test_filesystem_check(StorageIntegration::Microsoft).await } #[cfg(feature = "gcs")] #[tokio::test] #[serial] async fn test_filesystem_check_gcp() -> TestResult { - Ok(test_filesystem_check(StorageIntegration::Google).await?) + test_filesystem_check(StorageIntegration::Google).await } #[cfg(feature = "hdfs")] @@ -73,7 +73,7 @@ async fn test_filesystem_check(storage: StorageIntegration) -> TestResult { assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table.state.all_tombstones().get(file).unwrap(); - assert_eq!(remove.data_change, true); + assert!(remove.data_change); // An additonal run should return an empty list of orphaned actions let op = DeltaOps::from(table); @@ -114,7 +114,7 @@ async fn test_filesystem_check_partitioned() -> TestResult { assert_eq!(vec![file.to_string()], metrics.files_removed); let remove = table.state.all_tombstones().get(file).unwrap(); - assert_eq!(remove.data_change, true); + assert!(remove.data_change); Ok(()) } @@ -170,7 +170,7 @@ async fn test_filesystem_check_outdated() -> TestResult { if let Err(DeltaTableError::VersionAlreadyExists(version)) = res { assert!(version == 3); } else { - assert!(false); + panic!(); } Ok(()) diff --git a/rust/tests/command_optimize.rs b/rust/tests/command_optimize.rs index 2d31e330a4..153d7f86d5 100644 --- a/rust/tests/command_optimize.rs +++ b/rust/tests/command_optimize.rs @@ -6,11 +6,11 @@ use std::{collections::HashMap, error::Error, sync::Arc}; use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use arrow_select::concat::concat_batches; -use deltalake::action::{Action, DeltaOperation, Remove}; use deltalake::errors::DeltaTableError; use deltalake::operations::optimize::{create_merge_plan, MetricDetails, Metrics, OptimizeType}; use deltalake::operations::transaction::commit; use deltalake::operations::DeltaOps; +use deltalake::protocol::{Action, DeltaOperation, Remove}; use deltalake::storage::ObjectStoreRef; use deltalake::writer::{DeltaWriter, RecordBatchWriter}; use deltalake::{DeltaTable, PartitionFilter, Path, SchemaDataType, SchemaField}; diff --git a/rust/tests/command_restore.rs b/rust/tests/command_restore.rs index cda9d5a257..8f8ac11ca1 100644 --- a/rust/tests/command_restore.rs +++ b/rust/tests/command_restore.rs @@ -4,7 +4,7 @@ use arrow::datatypes::Schema as ArrowSchema; use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType, Field}; use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; -use deltalake::action::SaveMode; +use deltalake::protocol::SaveMode; use deltalake::{DeltaOps, DeltaTable, SchemaDataType, SchemaField}; use rand::Rng; use std::collections::HashMap; diff --git a/rust/tests/commit_info_format.rs b/rust/tests/commit_info_format.rs index c50a40d818..fdb1f89d92 100644 --- a/rust/tests/commit_info_format.rs +++ b/rust/tests/commit_info_format.rs @@ -1,8 +1,8 @@ #![allow(dead_code)] mod fs_common; -use deltalake::action::{Action, DeltaOperation, SaveMode}; use deltalake::operations::transaction::commit; +use deltalake::protocol::{Action, DeltaOperation, SaveMode}; use serde_json::json; use std::error::Error; use tempdir::TempDir; diff --git a/rust/tests/common/mod.rs b/rust/tests/common/mod.rs index b54fae20d0..2ba20d0635 100644 --- a/rust/tests/common/mod.rs +++ b/rust/tests/common/mod.rs @@ -1,9 +1,9 @@ #![allow(dead_code, unused_variables)] use bytes::Bytes; -use deltalake::action::{self, Add, DeltaOperation, Remove, SaveMode}; use deltalake::operations::create::CreateBuilder; use deltalake::operations::transaction::commit; +use deltalake::protocol::{self, Add, DeltaOperation, Remove, SaveMode}; use deltalake::storage::DeltaObjectStore; use deltalake::DeltaTableBuilder; use deltalake::{DeltaTable, Schema}; @@ -140,7 +140,7 @@ pub async fn add_file( partition_by: None, predicate: None, }; - let actions = vec![action::Action::add(add)]; + let actions = vec![protocol::Action::add(add)]; commit( table.object_store().as_ref(), &actions, @@ -173,7 +173,7 @@ pub async fn remove_file( ..Default::default() }; let operation = DeltaOperation::Delete { predicate: None }; - let actions = vec![action::Action::remove(remove)]; + let actions = vec![protocol::Action::remove(remove)]; commit( table.object_store().as_ref(), &actions, diff --git a/rust/tests/fs_common/mod.rs b/rust/tests/fs_common/mod.rs index 566436ac9c..3c5ab39e2c 100644 --- a/rust/tests/fs_common/mod.rs +++ b/rust/tests/fs_common/mod.rs @@ -1,7 +1,7 @@ use chrono::Utc; -use deltalake::action::{Action, Add, DeltaOperation, Remove, SaveMode}; use deltalake::operations::create::CreateBuilder; use deltalake::operations::transaction::commit; +use deltalake::protocol::{Action, Add, DeltaOperation, Remove, SaveMode}; use deltalake::storage::{DeltaObjectStore, GetResult, ObjectStoreResult}; use deltalake::{DeltaTable, Schema, SchemaDataType, SchemaField}; use object_store::path::Path as StorePath; diff --git a/rust/tests/integration_concurrent_writes.rs b/rust/tests/integration_concurrent_writes.rs index 314e9ce9a6..f34feac6e0 100644 --- a/rust/tests/integration_concurrent_writes.rs +++ b/rust/tests/integration_concurrent_writes.rs @@ -1,8 +1,8 @@ #![cfg(feature = "integration_test")] -use deltalake::action::{Action, Add, DeltaOperation, SaveMode}; use deltalake::operations::transaction::commit; use deltalake::operations::DeltaOps; +use deltalake::protocol::{Action, Add, DeltaOperation, SaveMode}; use deltalake::test_utils::{IntegrationContext, StorageIntegration, TestResult, TestTables}; use deltalake::{DeltaTable, DeltaTableBuilder, Schema, SchemaDataType, SchemaField}; use std::collections::HashMap; @@ -103,12 +103,12 @@ where assert_eq!(map.len() as i64, WORKERS * COMMITS); // check that we have unique and ascending versions committed - let mut versions = Vec::from_iter(map.keys().map(|x| x.clone())); + let mut versions = Vec::from_iter(map.keys().copied()); versions.sort(); assert_eq!(versions, Vec::from_iter(1i64..=WORKERS * COMMITS)); // check that each file for each worker is committed as expected - let mut files = Vec::from_iter(map.values().map(|x| x.clone())); + let mut files = Vec::from_iter(map.values().cloned()); files.sort(); let mut expected = Vec::new(); for w in 0..WORKERS { diff --git a/rust/tests/integration_datafusion.rs b/rust/tests/integration_datafusion.rs index 68c9149ac7..5aafe52e87 100644 --- a/rust/tests/integration_datafusion.rs +++ b/rust/tests/integration_datafusion.rs @@ -31,9 +31,9 @@ use datafusion_proto::bytes::{ }; use url::Url; -use deltalake::action::SaveMode; use deltalake::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; use deltalake::operations::create::CreateBuilder; +use deltalake::protocol::SaveMode; use deltalake::storage::DeltaObjectStore; use deltalake::writer::{DeltaWriter, RecordBatchWriter}; use deltalake::{ @@ -51,7 +51,7 @@ mod local { #[tokio::test] #[serial] async fn test_datafusion_local() -> TestResult { - Ok(test_datafusion(StorageIntegration::Local).await?) + test_datafusion(StorageIntegration::Local).await } fn get_scanned_files(node: &dyn ExecutionPlan) -> HashSet