From ca2acb85716a9835008f74929425cbba525d0a84 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Wed, 15 Nov 2023 01:08:08 +0100 Subject: [PATCH 01/35] first version --- python/deltalake/_internal.pyi | 8 + python/deltalake/writer.py | 420 +++++++++++++++++---------------- python/src/lib.rs | 53 +++++ 3 files changed, 278 insertions(+), 203 deletions(-) diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 4662f52f2f..f207c2276b 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -140,6 +140,14 @@ def write_new_deltalake( configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... +def write_to_deltalake( + table_uri: str, + data: pyarrow.RecordBatchReader, + partition_by: List[str], + mode: str, + max_rows_per_group: int, + storage_options: Optional[Dict[str, str]], +) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... # Can't implement inheritance (see note in src/schema.rs), so this is next diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index ef4ae3a57b..fa8e1f4b2f 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -39,6 +39,7 @@ from ._internal import DeltaDataChecker as _DeltaDataChecker from ._internal import batch_distinct from ._internal import write_new_deltalake as _write_new_deltalake +from ._internal import write_to_deltalake as _write_to_deltalake from .exceptions import DeltaProtocolError, TableNotFoundError from .table import MAX_SUPPORTED_WRITER_VERSION, DeltaTable @@ -90,6 +91,7 @@ def write_deltalake( storage_options: Optional[Dict[str, str]] = None, partition_filters: Optional[List[Tuple[str, str, Any]]] = None, large_dtypes: bool = False, + engine: Literal['pyarrow', 'rust'] = 'pyarrow', ) -> None: """Write to a Delta Lake table @@ -160,236 +162,248 @@ def write_deltalake( partition_filters: the partition filters that will be used for partition overwrite. large_dtypes: If True, the table schema is checked against large_dtypes """ - if _has_pandas and isinstance(data, pd.DataFrame): - if schema is not None: - data = pa.Table.from_pandas(data, schema=schema) - else: - data, schema = delta_arrow_schema_from_pandas(data) - table, table_uri = try_get_table_and_table_uri(table_or_uri, storage_options) + if table is not None: + storage_options = table._storage_options or {} + storage_options.update(storage_options or {}) - # We need to write against the latest table version - if table: - table.update_incremental() + + if engine == 'pyarrow': + + if _has_pandas and isinstance(data, pd.DataFrame): + if schema is not None: + data = pa.Table.from_pandas(data, schema=schema) + else: + data, schema = delta_arrow_schema_from_pandas(data) - if schema is None: - if isinstance(data, RecordBatchReader): - schema = data.schema - elif isinstance(data, Iterable): - raise ValueError("You must provide schema if data is Iterable") - else: - schema = data.schema + # We need to write against the latest table version + if table: + table.update_incremental() - if filesystem is not None: - raise NotImplementedError("Filesystem support is not yet implemented. #570") + if schema is None: + if isinstance(data, RecordBatchReader): + schema = data.schema + elif isinstance(data, Iterable): + raise ValueError("You must provide schema if data is Iterable") + else: + schema = data.schema - if table is not None: - storage_options = table._storage_options or {} - storage_options.update(storage_options or {}) + if filesystem is not None: + raise NotImplementedError("Filesystem support is not yet implemented. #570") - filesystem = pa_fs.PyFileSystem(DeltaStorageHandler(table_uri, storage_options)) - __enforce_append_only(table=table, configuration=configuration, mode=mode) - if isinstance(partition_by, str): - partition_by = [partition_by] + filesystem = pa_fs.PyFileSystem(DeltaStorageHandler(table_uri, storage_options)) - if table: # already exists - if schema != table.schema().to_pyarrow(as_large_types=large_dtypes) and not ( - mode == "overwrite" and overwrite_schema - ): - raise ValueError( - "Schema of data does not match table schema\n" - f"Data schema:\n{schema}\nTable Schema:\n{table.schema().to_pyarrow(as_large_types=large_dtypes)}" - ) + __enforce_append_only(table=table, configuration=configuration, mode=mode) - if mode == "error": - raise AssertionError("DeltaTable already exists.") - elif mode == "ignore": - return + if isinstance(partition_by, str): + partition_by = [partition_by] - current_version = table.version() + if table: # already exists + if schema != table.schema().to_pyarrow(as_large_types=large_dtypes) and not ( + mode == "overwrite" and overwrite_schema + ): + raise ValueError( + "Schema of data does not match table schema\n" + f"Data schema:\n{schema}\nTable Schema:\n{table.schema().to_pyarrow(as_large_types=large_dtypes)}" + ) - if partition_by: - assert partition_by == table.metadata().partition_columns - else: - partition_by = table.metadata().partition_columns + if mode == "error": + raise AssertionError("DeltaTable already exists.") + elif mode == "ignore": + return - if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: - raise DeltaProtocolError( - "This table's min_writer_version is " - f"{table.protocol().min_writer_version}, " - "but this method only supports version 2." - ) - else: # creating a new table - current_version = -1 + current_version = table.version() - dtype_map = { - pa.large_string(): pa.string(), # type: ignore - } + if partition_by: + assert partition_by == table.metadata().partition_columns + else: + partition_by = table.metadata().partition_columns - def _large_to_normal_dtype(dtype: pa.DataType) -> pa.DataType: - try: - return dtype_map[dtype] - except KeyError: - return dtype - - if partition_by: - if PYARROW_MAJOR_VERSION < 12: - partition_schema = pa.schema( - [ - pa.field(name, _large_to_normal_dtype(schema.field(name).type)) - for name in partition_by - ] - ) - else: - partition_schema = pa.schema([schema.field(name) for name in partition_by]) - partitioning = ds.partitioning(partition_schema, flavor="hive") - else: - partitioning = None + if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: + raise DeltaProtocolError( + "This table's min_writer_version is " + f"{table.protocol().min_writer_version}, " + "but this method only supports version 2." + ) + else: # creating a new table + current_version = -1 - add_actions: List[AddAction] = [] + dtype_map = { + pa.large_string(): pa.string(), # type: ignore + } - def visitor(written_file: Any) -> None: - path, partition_values = get_partitions_from_path(written_file.path) - stats = get_file_stats_from_metadata(written_file.metadata) + def _large_to_normal_dtype(dtype: pa.DataType) -> pa.DataType: + try: + return dtype_map[dtype] + except KeyError: + return dtype - # PyArrow added support for written_file.size in 9.0.0 - if PYARROW_MAJOR_VERSION >= 9: - size = written_file.size - elif filesystem is not None: - size = filesystem.get_file_info([path])[0].size + if partition_by: + if PYARROW_MAJOR_VERSION < 12: + partition_schema = pa.schema( + [ + pa.field(name, _large_to_normal_dtype(schema.field(name).type)) + for name in partition_by + ] + ) + else: + partition_schema = pa.schema([schema.field(name) for name in partition_by]) + partitioning = ds.partitioning(partition_schema, flavor="hive") else: - size = 0 - - add_actions.append( - AddAction( - path, - size, - partition_values, - int(datetime.now().timestamp() * 1000), - True, - json.dumps(stats, cls=DeltaJSONEncoder), + partitioning = None + + add_actions: List[AddAction] = [] + + def visitor(written_file: Any) -> None: + 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 + if PYARROW_MAJOR_VERSION >= 9: + size = written_file.size + elif filesystem is not None: + size = filesystem.get_file_info([path])[0].size + else: + size = 0 + + add_actions.append( + AddAction( + path, + size, + partition_values, + int(datetime.now().timestamp() * 1000), + True, + json.dumps(stats, cls=DeltaJSONEncoder), + ) ) - ) - if table is not None: - # We don't currently provide a way to set invariants - # (and maybe never will), so only enforce if already exist. - invariants = table.schema().invariants - checker = _DeltaDataChecker(invariants) - - def check_data_is_aligned_with_partition_filtering( - batch: pa.RecordBatch, - ) -> None: - if table is None: - return - existed_partitions: FrozenSet[ - FrozenSet[Tuple[str, Optional[str]]] - ] = table._table.get_active_partitions() - allowed_partitions: FrozenSet[ - FrozenSet[Tuple[str, Optional[str]]] - ] = table._table.get_active_partitions(partition_filters) - partition_values = pa.RecordBatch.from_arrays( - [ - batch.column(column_name) - for column_name in table.metadata().partition_columns - ], - table.metadata().partition_columns, + if table is not None: + # We don't currently provide a way to set invariants + # (and maybe never will), so only enforce if already exist. + invariants = table.schema().invariants + checker = _DeltaDataChecker(invariants) + + def check_data_is_aligned_with_partition_filtering( + batch: pa.RecordBatch, + ) -> None: + if table is None: + return + existed_partitions: FrozenSet[ + FrozenSet[Tuple[str, Optional[str]]] + ] = table._table.get_active_partitions() + allowed_partitions: FrozenSet[ + FrozenSet[Tuple[str, Optional[str]]] + ] = table._table.get_active_partitions(partition_filters) + partition_values = pa.RecordBatch.from_arrays( + [ + batch.column(column_name) + for column_name in table.metadata().partition_columns + ], + table.metadata().partition_columns, + ) + partition_values = batch_distinct(partition_values) + for i in range(partition_values.num_rows): + # Map will maintain order of partition_columns + partition_map = { + column_name: encode_partition_value( + batch.column(column_name)[i].as_py() + ) + for column_name in table.metadata().partition_columns + } + partition = frozenset(partition_map.items()) + if ( + partition not in allowed_partitions + and partition in existed_partitions + ): + partition_repr = " ".join( + f"{key}={value}" for key, value in partition_map.items() + ) + raise ValueError( + f"Data should be aligned with partitioning. " + f"Data contained values for partition {partition_repr}" + ) + + def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: + checker.check_batch(batch) + + if mode == "overwrite" and partition_filters: + check_data_is_aligned_with_partition_filtering(batch) + + return batch + + if isinstance(data, RecordBatchReader): + batch_iter = data + elif isinstance(data, pa.RecordBatch): + batch_iter = [data] + elif isinstance(data, pa.Table): + batch_iter = data.to_batches() + elif isinstance(data, ds.Dataset): + batch_iter = data.to_batches() + else: + batch_iter = data + + data = RecordBatchReader.from_batches( + schema, (validate_batch(batch) for batch in batch_iter) ) - partition_values = batch_distinct(partition_values) - for i in range(partition_values.num_rows): - # Map will maintain order of partition_columns - partition_map = { - column_name: encode_partition_value( - batch.column(column_name)[i].as_py() - ) - for column_name in table.metadata().partition_columns - } - partition = frozenset(partition_map.items()) - if ( - partition not in allowed_partitions - and partition in existed_partitions - ): - partition_repr = " ".join( - f"{key}={value}" for key, value in partition_map.items() - ) - raise ValueError( - f"Data should be aligned with partitioning. " - f"Data contained values for partition {partition_repr}" - ) - - def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: - checker.check_batch(batch) - - if mode == "overwrite" and partition_filters: - check_data_is_aligned_with_partition_filtering(batch) - - return batch - - if isinstance(data, RecordBatchReader): - batch_iter = data - elif isinstance(data, pa.RecordBatch): - batch_iter = [data] - elif isinstance(data, pa.Table): - batch_iter = data.to_batches() - elif isinstance(data, ds.Dataset): - batch_iter = data.to_batches() - else: - batch_iter = data - data = RecordBatchReader.from_batches( - schema, (validate_batch(batch) for batch in batch_iter) - ) + if file_options is not None: + file_options.update(use_compliant_nested_type=False) + else: + file_options = ds.ParquetFileFormat().make_write_options( + use_compliant_nested_type=False + ) - if file_options is not None: - file_options.update(use_compliant_nested_type=False) - else: - file_options = ds.ParquetFileFormat().make_write_options( - use_compliant_nested_type=False + ds.write_dataset( + data, + base_dir="/", + basename_template=f"{current_version + 1}-{uuid.uuid4()}-{{i}}.parquet", + format="parquet", + partitioning=partitioning, + # It will not accept a schema if using a RBR + schema=schema if not isinstance(data, RecordBatchReader) else None, + file_visitor=visitor, + existing_data_behavior="overwrite_or_ignore", + file_options=file_options, + max_open_files=max_open_files, + max_rows_per_file=max_rows_per_file, + min_rows_per_group=min_rows_per_group, + max_rows_per_group=max_rows_per_group, + filesystem=filesystem, + max_partitions=max_partitions, ) - ds.write_dataset( - data, - base_dir="/", - basename_template=f"{current_version + 1}-{uuid.uuid4()}-{{i}}.parquet", - format="parquet", - partitioning=partitioning, - # It will not accept a schema if using a RBR - schema=schema if not isinstance(data, RecordBatchReader) else None, - file_visitor=visitor, - existing_data_behavior="overwrite_or_ignore", - file_options=file_options, - max_open_files=max_open_files, - max_rows_per_file=max_rows_per_file, - min_rows_per_group=min_rows_per_group, - max_rows_per_group=max_rows_per_group, - filesystem=filesystem, - max_partitions=max_partitions, - ) - - if table is None: - _write_new_deltalake( - table_uri, - schema, - add_actions, - mode, - partition_by or [], - name, - description, - configuration, - storage_options, - ) + if table is None: + _write_new_deltalake( + table_uri, + schema, + add_actions, + mode, + partition_by or [], + name, + description, + configuration, + storage_options, + ) + else: + table._table.create_write_transaction( + add_actions, + mode, + partition_by or [], + schema, + partition_filters, + ) + table.update_incremental() else: - table._table.create_write_transaction( - add_actions, - mode, - partition_by or [], - schema, - partition_filters, - ) - table.update_incremental() - + _write_to_deltalake( + table_uri=table_uri, + data=data.to_reader(), + partition_by=partition_by, + mode=mode, + max_rows_per_group=max_rows_per_group, + storage_options=storage_options + ) def __enforce_append_only( table: Optional[DeltaTable], diff --git a/python/src/lib.rs b/python/src/lib.rs index 5ee72f72d0..0ff972c35e 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1135,6 +1135,58 @@ impl From<&PyAddAction> for Add { } } +#[pyfunction] +#[allow(clippy::too_many_arguments)] +fn write_to_deltalake( + table_uri: String, + data: PyArrowType, + // schema: Option>, + partition_by: Vec, // or Vec + mode: String, + // max_partitions: i64, + // max_rows_per_file: i64, + // min_rows_per_group: i64, + max_rows_per_group: i64, + // name: Option, + // description: Option, + // configuration: Option>>, + // overwrite_schema: bool, + storage_options: Option>, +) -> PyResult<()> { + + // let schema = data.0.schema(); + let batches = data.0.map(|batch| batch.unwrap()).collect::>(); + // let batches = data.0; + + let mode = save_mode_from_str(&mode)?; + // let new_schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; + + // let existing_schema = self._table.get_schema().map_err(PythonError::from)?; + + // let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; + + + let options = storage_options.clone().unwrap_or_default(); + let table = DeltaTableBuilder::from_uri(table_uri).with_storage_options(options); + + let table = table.build().map_err(PythonError::from)?; + + let builder = DeltaOps(table) + .write(batches) + .with_save_mode(mode) + .with_write_batch_size(max_rows_per_group as usize) + .with_partition_columns(partition_by); + + rt()? + .block_on(builder.into_future()) + .map_err(PythonError::from)?; + + Ok(()) +} + + + + #[pyfunction] #[allow(clippy::too_many_arguments)] fn write_new_deltalake( @@ -1225,6 +1277,7 @@ fn _internal(py: Python, m: &PyModule) -> PyResult<()> { m.add("__version__", env!("CARGO_PKG_VERSION"))?; m.add_function(pyo3::wrap_pyfunction!(rust_core_version, m)?)?; m.add_function(pyo3::wrap_pyfunction!(write_new_deltalake, m)?)?; + m.add_function(pyo3::wrap_pyfunction!(write_to_deltalake, m)?)?; m.add_function(pyo3::wrap_pyfunction!(batch_distinct, m)?)?; m.add_class::()?; m.add_class::()?; From 3f55470a77e7d2c4a5da327e1d2bc507529ca5f9 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Wed, 15 Nov 2023 20:02:01 +0100 Subject: [PATCH 02/35] add try from uri with storage options --- crates/deltalake-core/src/operations/mod.rs | 13 +++++++++++++ python/src/lib.rs | 7 ++----- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index a0dbfd0239..557a4b9dd1 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -7,6 +7,7 @@ //! with a [data stream][datafusion::physical_plan::SendableRecordBatchStream], //! if the operation returns data as well. +use std::collections::{HashMap}; use self::create::CreateBuilder; use self::filesystem_check::FileSystemCheckBuilder; use self::vacuum::VacuumBuilder; @@ -72,6 +73,18 @@ impl DeltaOps { Err(err) => Err(err), } } + + + /// try from uri with storage options + pub async fn try_from_uri_with_storage_options(uri: impl AsRef, storage_options: HashMap) -> DeltaResult { + let mut table = DeltaTableBuilder::from_uri(uri).with_storage_options(storage_options).build()?; + // We allow for uninitialized locations, since we may want to create the table + match table.load().await { + Ok(_) => Ok(table.into()), + Err(DeltaTableError::NotATable(_)) => Ok(table.into()), + Err(err) => Err(err), + } + } /// Create a new [`DeltaOps`] instance, backed by an un-initialized in memory table /// diff --git a/python/src/lib.rs b/python/src/lib.rs index 0ff972c35e..4d93d687d5 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1165,13 +1165,10 @@ fn write_to_deltalake( // let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; - let options = storage_options.clone().unwrap_or_default(); - let table = DeltaTableBuilder::from_uri(table_uri).with_storage_options(options); - - let table = table.build().map_err(PythonError::from)?; + let table = rt()?.block_on(DeltaOps::try_from_uri_with_storage_options(&table_uri, options)).map_err(PythonError::from)?; - let builder = DeltaOps(table) + let builder = table .write(batches) .with_save_mode(mode) .with_write_batch_size(max_rows_per_group as usize) From 714cc561d9572c8e344cc6718f94d1190ddf6fd0 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Wed, 15 Nov 2023 20:41:18 +0100 Subject: [PATCH 03/35] Start to enable overwrite_schema --- crates/deltalake-core/src/operations/write.rs | 13 +++++++-- python/deltalake/_internal.pyi | 1 + python/deltalake/writer.py | 28 ++++++++++--------- python/src/lib.rs | 3 +- 4 files changed, 29 insertions(+), 16 deletions(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index cb68b72bb2..fdc6f82faa 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -103,6 +103,8 @@ pub struct WriteBuilder { write_batch_size: Option, /// RecordBatches to be written into the table batches: Option>, + /// whether to overwrite the schema + overwrite_schema: bool, /// how to handle cast failures, either return NULL (safe=true) or return ERR (safe=false) safe_cast: bool, /// Parquet writer properties @@ -126,6 +128,7 @@ impl WriteBuilder { write_batch_size: None, batches: None, safe_cast: false, + overwrite_schema: false, writer_properties: None, app_metadata: None, } @@ -137,6 +140,12 @@ impl WriteBuilder { self } + /// Add overwrite_schema + pub fn with_overwrite_schema(mut self, overwrite_schema: bool) -> Self { + self.overwrite_schema = overwrite_schema; + self + } + /// When using `Overwrite` mode, replace data that matches a predicate pub fn with_replace_where(mut self, predicate: impl Into) -> Self { self.predicate = Some(predicate.into()); @@ -353,9 +362,9 @@ impl std::future::IntoFuture for WriteBuilder { .or_else(|_| this.snapshot.arrow_schema()) .unwrap_or(schema.clone()); - if !can_cast_batch(schema.fields(), table_schema.fields()) { + if !can_cast_batch(schema.fields(), table_schema.fields()) && !this.overwrite_schema { return Err(DeltaTableError::Generic( - "Updating table schema not yet implemented".to_string(), + "Schema of data does not match table schema".to_string(), )); }; diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index f207c2276b..f9086d3329 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -146,6 +146,7 @@ def write_to_deltalake( partition_by: List[str], mode: str, max_rows_per_group: int, + overwrite_schema: bool, storage_options: Optional[Dict[str, str]], ) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index fa8e1f4b2f..b58287e10d 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -91,7 +91,7 @@ def write_deltalake( storage_options: Optional[Dict[str, str]] = None, partition_filters: Optional[List[Tuple[str, str, Any]]] = None, large_dtypes: bool = False, - engine: Literal['pyarrow', 'rust'] = 'pyarrow', + engine: Literal["pyarrow", "rust"] = "pyarrow", ) -> None: """Write to a Delta Lake table @@ -167,9 +167,7 @@ def write_deltalake( storage_options = table._storage_options or {} storage_options.update(storage_options or {}) - - if engine == 'pyarrow': - + if engine == "pyarrow": if _has_pandas and isinstance(data, pd.DataFrame): if schema is not None: data = pa.Table.from_pandas(data, schema=schema) @@ -189,9 +187,9 @@ def write_deltalake( schema = data.schema if filesystem is not None: - raise NotImplementedError("Filesystem support is not yet implemented. #570") - - + raise NotImplementedError( + "Filesystem support is not yet implemented. #570" + ) filesystem = pa_fs.PyFileSystem(DeltaStorageHandler(table_uri, storage_options)) @@ -201,9 +199,9 @@ def write_deltalake( partition_by = [partition_by] if table: # already exists - if schema != table.schema().to_pyarrow(as_large_types=large_dtypes) and not ( - mode == "overwrite" and overwrite_schema - ): + if schema != table.schema().to_pyarrow( + as_large_types=large_dtypes + ) and not (mode == "overwrite" and overwrite_schema): raise ValueError( "Schema of data does not match table schema\n" f"Data schema:\n{schema}\nTable Schema:\n{table.schema().to_pyarrow(as_large_types=large_dtypes)}" @@ -249,7 +247,9 @@ def _large_to_normal_dtype(dtype: pa.DataType) -> pa.DataType: ] ) else: - partition_schema = pa.schema([schema.field(name) for name in partition_by]) + partition_schema = pa.schema( + [schema.field(name) for name in partition_by] + ) partitioning = ds.partitioning(partition_schema, flavor="hive") else: partitioning = None @@ -402,8 +402,10 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: partition_by=partition_by, mode=mode, max_rows_per_group=max_rows_per_group, - storage_options=storage_options - ) + overwrite_schema=overwrite_schema, + storage_options=storage_options, + ) + def __enforce_append_only( table: Optional[DeltaTable], diff --git a/python/src/lib.rs b/python/src/lib.rs index 4d93d687d5..799b82caad 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1150,7 +1150,7 @@ fn write_to_deltalake( // name: Option, // description: Option, // configuration: Option>>, - // overwrite_schema: bool, + overwrite_schema: bool, storage_options: Option>, ) -> PyResult<()> { @@ -1171,6 +1171,7 @@ fn write_to_deltalake( let builder = table .write(batches) .with_save_mode(mode) + .with_overwrite_schema(overwrite_schema) .with_write_batch_size(max_rows_per_group as usize) .with_partition_columns(partition_by); From 52f0d6a8903f5ed82cd869bdf02a078e14067606 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Thu, 16 Nov 2023 19:25:06 +0100 Subject: [PATCH 04/35] add tests to check rust py03 writer --- crates/deltalake-core/src/operations/mod.rs | 4 +- python/deltalake/_internal.pyi | 5 +- python/deltalake/writer.py | 75 +++++--- python/src/lib.rs | 64 ++++--- python/tests/test_writer.py | 179 ++++++++++++++++---- 5 files changed, 247 insertions(+), 80 deletions(-) diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index 557a4b9dd1..5eef40698f 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -77,7 +77,9 @@ impl DeltaOps { /// try from uri with storage options pub async fn try_from_uri_with_storage_options(uri: impl AsRef, storage_options: HashMap) -> DeltaResult { - let mut table = DeltaTableBuilder::from_uri(uri).with_storage_options(storage_options).build()?; + let mut table = DeltaTableBuilder::from_uri(uri) + .with_storage_options(storage_options) + .build()?; // We allow for uninitialized locations, since we may want to create the table match table.load().await { Ok(_) => Ok(table.into()), diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index f9086d3329..2a4f77993c 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -143,10 +143,13 @@ def write_new_deltalake( def write_to_deltalake( table_uri: str, data: pyarrow.RecordBatchReader, - partition_by: List[str], + partition_by: Optional[List[str]], mode: str, max_rows_per_group: int, overwrite_schema: bool, + name: Optional[str], + description: Optional[str], + configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index b58287e10d..47651dd16b 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -167,17 +167,66 @@ def write_deltalake( storage_options = table._storage_options or {} storage_options.update(storage_options or {}) - if engine == "pyarrow": + __enforce_append_only(table=table, configuration=configuration, mode=mode) + + if isinstance(partition_by, str): + partition_by = [partition_by] + + if table: + table.update_incremental() + + if engine == "rust": + # Easier to do this check in Python than rust + if table is not None and mode == "ignore": + return + ### COMMENTS ### + # - Don't check partition columns if they are the same, this is done on the rust side implicility + # - Consolidate the recordbatch reader part with the new update + # - Add overwrite schema functionality in rust writer + # - Figure out how to add name, description and configuration to the correct metadata in transaction + + if isinstance(data, RecordBatchReader): + batch_iter = data + elif isinstance(data, pa.RecordBatch): + batch_iter = [data] + elif isinstance(data, pa.Table): + batch_iter = data.to_reader() + elif isinstance(data, ds.Dataset): + batch_iter = data.scanner().to_reader() + elif isinstance(data, pd.DataFrame): + batch_iter = pa.Table.from_pandas(data).to_reader() + else: + batch_iter = data + + if schema is None: + if isinstance(batch_iter, RecordBatchReader): + schema = batch_iter.schema + elif isinstance(batch_iter, Iterable): + raise ValueError("You must provide schema if data is Iterable") + + data = RecordBatchReader.from_batches(schema, (batch for batch in batch_iter)) + _write_to_deltalake( + table_uri=table_uri, + data=data, + partition_by=partition_by, + mode=mode, + max_rows_per_group=max_rows_per_group, + overwrite_schema=overwrite_schema, + name=name, + description=description, + configuration=configuration, + storage_options=storage_options, + ) + if table: + table.update_incremental() + + elif engine == "pyarrow": if _has_pandas and isinstance(data, pd.DataFrame): if schema is not None: data = pa.Table.from_pandas(data, schema=schema) else: data, schema = delta_arrow_schema_from_pandas(data) - # We need to write against the latest table version - if table: - table.update_incremental() - if schema is None: if isinstance(data, RecordBatchReader): schema = data.schema @@ -193,11 +242,6 @@ def write_deltalake( filesystem = pa_fs.PyFileSystem(DeltaStorageHandler(table_uri, storage_options)) - __enforce_append_only(table=table, configuration=configuration, mode=mode) - - if isinstance(partition_by, str): - partition_by = [partition_by] - if table: # already exists if schema != table.schema().to_pyarrow( as_large_types=large_dtypes @@ -206,7 +250,6 @@ def write_deltalake( "Schema of data does not match table schema\n" f"Data schema:\n{schema}\nTable Schema:\n{table.schema().to_pyarrow(as_large_types=large_dtypes)}" ) - if mode == "error": raise AssertionError("DeltaTable already exists.") elif mode == "ignore": @@ -396,15 +439,7 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: ) table.update_incremental() else: - _write_to_deltalake( - table_uri=table_uri, - data=data.to_reader(), - partition_by=partition_by, - mode=mode, - max_rows_per_group=max_rows_per_group, - overwrite_schema=overwrite_schema, - storage_options=storage_options, - ) + raise ValueError("Only `pyarrow` or `rust` are valid inputs for the engine.") def __enforce_append_only( diff --git a/python/src/lib.rs b/python/src/lib.rs index 799b82caad..5ded14c6be 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -43,6 +43,7 @@ use deltalake::DeltaTableBuilder; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::{PyFrozenSet, PyType}; +use serde_json::Value; use crate::error::DeltaProtocolError; use crate::error::PythonError; @@ -1140,41 +1141,58 @@ impl From<&PyAddAction> for Add { fn write_to_deltalake( table_uri: String, data: PyArrowType, - // schema: Option>, - partition_by: Vec, // or Vec + // schema: Option>, // maybe do the schema casting on python side mode: String, - // max_partitions: i64, - // max_rows_per_file: i64, - // min_rows_per_group: i64, max_rows_per_group: i64, - // name: Option, - // description: Option, - // configuration: Option>>, overwrite_schema: bool, + partition_by: Option>, + name: Option, + description: Option, + configuration: Option>>, storage_options: Option>, ) -> PyResult<()> { - - // let schema = data.0.schema(); let batches = data.0.map(|batch| batch.unwrap()).collect::>(); - // let batches = data.0; + let save_mode = save_mode_from_str(&mode)?; - let mode = save_mode_from_str(&mode)?; - // let new_schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; + let mut metadata: HashMap = HashMap::new(); - // let existing_schema = self._table.get_schema().map_err(PythonError::from)?; + if let Some(name) = name { + metadata.insert("name".to_string(), name.into()); + } + + if let Some(description) = description { + metadata.insert("description".to_string(), description.into()); + } - // let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; + if let Some(configuration) = configuration { + metadata.insert("configuration".to_string(), json!(configuration)); + } + // // This should be done when the table can not be loaded ... + // match save_mode { + // SaveMode::Ignore => { + // return Ok(()) + // } + // _ => () + // } let options = storage_options.clone().unwrap_or_default(); - let table = rt()?.block_on(DeltaOps::try_from_uri_with_storage_options(&table_uri, options)).map_err(PythonError::from)?; + let table = rt()? + .block_on(DeltaOps::try_from_uri_with_storage_options( + &table_uri, options, + )) + .map_err(PythonError::from)?; - let builder = table + let mut builder = table .write(batches) - .with_save_mode(mode) + .with_save_mode(save_mode) .with_overwrite_schema(overwrite_schema) - .with_write_batch_size(max_rows_per_group as usize) - .with_partition_columns(partition_by); - + .with_metadata(metadata) + .with_write_batch_size(max_rows_per_group as usize); + + if let Some(partition_columns) = partition_by { + builder = builder.with_partition_columns(partition_columns); + } + rt()? .block_on(builder.into_future()) .map_err(PythonError::from)?; @@ -1182,9 +1200,7 @@ fn write_to_deltalake( Ok(()) } - - - +use serde_json::json; #[pyfunction] #[allow(clippy::too_many_arguments)] fn write_new_deltalake( diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index d048f8b79b..d2a2abefa6 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -6,7 +6,7 @@ import threading from datetime import date, datetime from math import inf -from typing import Any, Dict, Iterable, List +from typing import Any, Dict, Iterable, List, Literal from unittest.mock import Mock import pyarrow as pa @@ -17,7 +17,7 @@ from pyarrow.lib import RecordBatchReader from deltalake import DeltaTable, write_deltalake -from deltalake.exceptions import CommitFailedError, DeltaProtocolError +from deltalake.exceptions import CommitFailedError, DeltaError, DeltaProtocolError from deltalake.table import ProtocolVersions from deltalake.writer import try_get_table_and_table_uri @@ -29,24 +29,30 @@ _has_pandas = True +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) @pytest.mark.skip(reason="Waiting on #570") -def test_handle_existing(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_handle_existing( + tmp_path: pathlib.Path, sample_data: pa.Table, engine: Literal["pyarrow", "rust"] +): # if uri points to a non-empty directory that isn't a delta table, error tmp_path p = tmp_path / "hello.txt" p.write_text("hello") with pytest.raises(OSError) as exception: - write_deltalake(tmp_path, sample_data, mode="overwrite") + write_deltalake(tmp_path, sample_data, mode="overwrite", engine=engine) assert "directory is not empty" in str(exception) -def test_roundtrip_basic(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_roundtrip_basic( + tmp_path: pathlib.Path, sample_data: pa.Table, engine: Literal["pyarrow", "rust"] +): # Check we can create the subdirectory tmp_path = tmp_path / "path" / "to" / "table" start_time = datetime.now().timestamp() - write_deltalake(tmp_path, sample_data) + write_deltalake(tmp_path, sample_data, engine=engine) end_time = datetime.now().timestamp() assert ("0" * 20 + ".json") in os.listdir(tmp_path / "_delta_log") @@ -71,7 +77,8 @@ def test_roundtrip_basic(tmp_path: pathlib.Path, sample_data: pa.Table): assert modification_time < end_time -def test_roundtrip_nulls(tmp_path: pathlib.Path): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_roundtrip_nulls(tmp_path: pathlib.Path, engine: Literal["pyarrow", "rust"]): data = pa.table({"x": pa.array([None, None, 1, 2], type=pa.int64())}) # One row group will have values, one will be all nulls. # The first will have None in min and max stats, so we need to handle that. @@ -91,6 +98,7 @@ def test_roundtrip_nulls(tmp_path: pathlib.Path): min_rows_per_group=2, max_rows_per_group=2, mode="overwrite", + engine=engine, ) delta_table = DeltaTable(tmp_path) @@ -105,11 +113,23 @@ def test_enforce_schema(existing_table: DeltaTable, mode: str): bad_data = pa.table({"x": pa.array([1, 2, 3])}) with pytest.raises(ValueError): - write_deltalake(existing_table, bad_data, mode=mode) + write_deltalake(existing_table, bad_data, mode=mode, engine="pyarrow") table_uri = existing_table._table.table_uri() with pytest.raises(ValueError): - write_deltalake(table_uri, bad_data, mode=mode) + write_deltalake(table_uri, bad_data, mode=mode, engine="pyarrow") + + +@pytest.mark.parametrize("mode", ["append", "overwrite"]) +def test_enforce_schema_rust_writer(existing_table: DeltaTable, mode: str): + bad_data = pa.table({"x": pa.array([1, 2, 3])}) + + with pytest.raises(DeltaError): + write_deltalake(existing_table, bad_data, mode=mode, engine="rust") + + table_uri = existing_table._table.table_uri() + with pytest.raises(DeltaError): + write_deltalake(table_uri, bad_data, mode=mode, engine="rust") def test_update_schema(existing_table: DeltaTable): @@ -125,12 +145,39 @@ def test_update_schema(existing_table: DeltaTable): assert existing_table.schema().to_pyarrow() == new_data.schema -def test_local_path(tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch): +# def test_update_schema_rust_writer(existing_table: DeltaTable): # Test fails +# new_data = pa.table({"x": pa.array([1, 2, 3])}) + +# with pytest.raises(DeltaError): +# write_deltalake( +# existing_table, +# new_data, +# mode="append", +# overwrite_schema=True, +# engine="rust", +# ) + +# write_deltalake( +# existing_table, new_data, mode="overwrite", overwrite_schema=True, engine="rust" +# ) + +# read_data = existing_table.to_pyarrow_table() +# assert new_data == read_data +# assert existing_table.schema().to_pyarrow() == new_data.schema + + +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_local_path( + tmp_path: pathlib.Path, + sample_data: pa.Table, + monkeypatch, + engine: Literal["pyarrow", "rust"], +): monkeypatch.chdir(tmp_path) # Make tmp_path the working directory (tmp_path / "path/to/table").mkdir(parents=True) local_path = "./path/to/table" - write_deltalake(local_path, sample_data) + write_deltalake(local_path, sample_data, engine=engine) delta_table = DeltaTable(local_path) assert delta_table.schema().to_pyarrow() == sample_data.schema @@ -138,13 +185,15 @@ def test_local_path(tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch): assert table == sample_data -def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table, engine): write_deltalake( tmp_path, sample_data, name="test_name", description="test_desc", configuration={"delta.appendOnly": "false", "foo": "bar"}, + engine=engine, ) delta_table = DeltaTable(tmp_path) @@ -156,6 +205,7 @@ def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): assert metadata.configuration == {"delta.appendOnly": "false", "foo": "bar"} +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) @pytest.mark.parametrize( "column", [ @@ -173,9 +223,9 @@ def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): ], ) def test_roundtrip_partitioned( - tmp_path: pathlib.Path, sample_data: pa.Table, column: str + tmp_path: pathlib.Path, sample_data: pa.Table, column: str, engine ): - write_deltalake(tmp_path, sample_data, partition_by=column) + write_deltalake(tmp_path, sample_data, partition_by=column, engine=engine) delta_table = DeltaTable(tmp_path) assert delta_table.schema().to_pyarrow() == sample_data.schema @@ -189,11 +239,16 @@ def test_roundtrip_partitioned( assert add_path.count("/") == 1 -def test_roundtrip_null_partition(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_roundtrip_null_partition( + tmp_path: pathlib.Path, sample_data: pa.Table, engine +): sample_data = sample_data.add_column( 0, "utf8_with_nulls", pa.array(["a"] * 4 + [None]) ) - write_deltalake(tmp_path, sample_data, partition_by=["utf8_with_nulls"]) + write_deltalake( + tmp_path, sample_data, partition_by=["utf8_with_nulls"], engine=engine + ) delta_table = DeltaTable(tmp_path) assert delta_table.schema().to_pyarrow() == sample_data.schema @@ -203,8 +258,13 @@ def test_roundtrip_null_partition(tmp_path: pathlib.Path, sample_data: pa.Table) assert table == sample_data -def test_roundtrip_multi_partitioned(tmp_path: pathlib.Path, sample_data: pa.Table): - write_deltalake(tmp_path, sample_data, partition_by=["int32", "bool"]) +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_roundtrip_multi_partitioned( + tmp_path: pathlib.Path, sample_data: pa.Table, engine +): + write_deltalake( + tmp_path, sample_data, partition_by=["int32", "bool"], engine=engine + ) delta_table = DeltaTable(tmp_path) assert delta_table.schema().to_pyarrow() == sample_data.schema @@ -236,7 +296,25 @@ def test_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data -def test_append_only_should_append_only_with_the_overwrite_mode( +def test_write_modes_rust(tmp_path: pathlib.Path, sample_data: pa.Table): + write_deltalake(tmp_path, sample_data) + assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data + + with pytest.raises(DeltaError): + write_deltalake(tmp_path, sample_data, mode="error", engine="rust") + + write_deltalake(tmp_path, sample_data, mode="ignore", engine="rust") + assert ("0" * 19 + "1.json") not in os.listdir(tmp_path / "_delta_log") + + write_deltalake(tmp_path, sample_data, mode="append", engine="rust") + expected = pa.concat_tables([sample_data, sample_data]) + assert DeltaTable(tmp_path).to_pyarrow_table() == expected + + write_deltalake(tmp_path, sample_data, mode="overwrite", engine="rust") + assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data + + +def test_append_only_should_append_only_with_the_overwrite_mode( # Create rust equivalent rust tmp_path: pathlib.Path, sample_data: pa.Table ): config = {"delta.appendOnly": "true"} @@ -265,8 +343,9 @@ def test_append_only_should_append_only_with_the_overwrite_mode( assert table.version() == 1 -def test_writer_with_table(existing_table: DeltaTable, sample_data: pa.Table): - write_deltalake(existing_table, sample_data, mode="overwrite") +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_writer_with_table(existing_table: DeltaTable, sample_data: pa.Table, engine): + write_deltalake(existing_table, sample_data, mode="overwrite", engine=engine) assert existing_table.to_pyarrow_table() == sample_data @@ -277,9 +356,25 @@ def test_fails_wrong_partitioning(existing_table: DeltaTable, sample_data: pa.Ta ) +def test_fails_wrong_partitioning_rust_writer( + existing_table: DeltaTable, sample_data: pa.Table +): + with pytest.raises(DeltaError): + write_deltalake( + existing_table, + sample_data, + mode="append", + partition_by="int32", + engine="rust", + ) + + +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) @pytest.mark.pandas @pytest.mark.parametrize("schema_provided", [True, False]) -def test_write_pandas(tmp_path: pathlib.Path, sample_data: pa.Table, schema_provided): +def test_write_pandas( + tmp_path: pathlib.Path, sample_data: pa.Table, schema_provided, engine +): # When timestamp is converted to Pandas, it gets casted to ns resolution, # but Delta Lake schemas only support us resolution. sample_pandas = sample_data.to_pandas() @@ -287,42 +382,52 @@ def test_write_pandas(tmp_path: pathlib.Path, sample_data: pa.Table, schema_prov schema = sample_data.schema else: schema = None - write_deltalake(tmp_path, sample_pandas, schema=schema) + write_deltalake(tmp_path, sample_pandas, schema=schema, engine=engine) delta_table = DeltaTable(tmp_path) df = delta_table.to_pandas() assert_frame_equal(df, sample_pandas) +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_write_iterator( - tmp_path: pathlib.Path, existing_table: DeltaTable, sample_data: pa.Table + tmp_path: pathlib.Path, existing_table: DeltaTable, sample_data: pa.Table, engine ): batches = existing_table.to_pyarrow_dataset().to_batches() with pytest.raises(ValueError): - write_deltalake(tmp_path, batches, mode="overwrite") + write_deltalake(tmp_path, batches, mode="overwrite", engine=engine) - write_deltalake(tmp_path, batches, schema=sample_data.schema, mode="overwrite") + write_deltalake( + tmp_path, batches, schema=sample_data.schema, mode="overwrite", engine=engine + ) assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_write_recordbatchreader( - tmp_path: pathlib.Path, existing_table: DeltaTable, sample_data: pa.Table + tmp_path: pathlib.Path, + existing_table: DeltaTable, + sample_data: pa.Table, + engine: Literal["pyarrow", "rust"], ): batches = existing_table.to_pyarrow_dataset().to_batches() reader = RecordBatchReader.from_batches( existing_table.to_pyarrow_dataset().schema, batches ) - write_deltalake(tmp_path, reader, mode="overwrite") + write_deltalake(tmp_path, reader, mode="overwrite", engine=engine) assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data -def test_writer_partitioning(tmp_path: pathlib.Path): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_writer_partitioning( + tmp_path: pathlib.Path, engine: Literal["pyarrow", "rust"] +): test_strings = ["a=b", "hello world", "hello%20world"] data = pa.table( {"p": pa.array(test_strings), "x": pa.array(range(len(test_strings)))} ) - write_deltalake(tmp_path, data) + write_deltalake(tmp_path, data, engine=engine) assert DeltaTable(tmp_path).to_pyarrow_table() == data @@ -411,7 +516,8 @@ def test_writer_stats(existing_table: DeltaTable, sample_data: pa.Table): assert stats["maxValues"] == expected_maxs -def test_writer_null_stats(tmp_path: pathlib.Path): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_writer_null_stats(tmp_path: pathlib.Path, engine: Literal["pyarrow", "rust"]): data = pa.table( { "int32": pa.array([1, None, 2, None], pa.int32()), @@ -419,7 +525,7 @@ def test_writer_null_stats(tmp_path: pathlib.Path): "str": pa.array([None] * 4, pa.string()), } ) - write_deltalake(tmp_path, data) + write_deltalake(tmp_path, data, engine=engine) table = DeltaTable(tmp_path) stats = get_stats(table) @@ -428,10 +534,15 @@ def test_writer_null_stats(tmp_path: pathlib.Path): assert stats["nullCount"] == expected_nulls -def test_writer_fails_on_protocol(existing_table: DeltaTable, sample_data: pa.Table): +@pytest.mark.parametrize("engine", ["pyarrow"]) # This one is broken +def test_writer_fails_on_protocol( + existing_table: DeltaTable, + sample_data: pa.Table, + engine: Literal["pyarrow", "rust"], +): existing_table.protocol = Mock(return_value=ProtocolVersions(1, 3)) with pytest.raises(DeltaProtocolError): - write_deltalake(existing_table, sample_data, mode="overwrite") + write_deltalake(existing_table, sample_data, mode="overwrite", engine=engine) @pytest.mark.parametrize( From c067d4de26467e6390e0ec56f01f006c554ce113 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Thu, 16 Nov 2023 19:35:38 +0100 Subject: [PATCH 05/35] remove comment --- python/deltalake/writer.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 47651dd16b..e69560e897 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -176,14 +176,8 @@ def write_deltalake( table.update_incremental() if engine == "rust": - # Easier to do this check in Python than rust if table is not None and mode == "ignore": return - ### COMMENTS ### - # - Don't check partition columns if they are the same, this is done on the rust side implicility - # - Consolidate the recordbatch reader part with the new update - # - Add overwrite schema functionality in rust writer - # - Figure out how to add name, description and configuration to the correct metadata in transaction if isinstance(data, RecordBatchReader): batch_iter = data From 5c5f2475e58ae571abc8c131b044af0a766c20aa Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Fri, 17 Nov 2023 10:28:08 +0100 Subject: [PATCH 06/35] rename and clean up --- python/deltalake/_internal.pyi | 6 ++-- python/deltalake/writer.py | 24 ++++++++----- python/src/lib.rs | 31 ++--------------- python/tests/test_writer.py | 63 +++++++++++++++++++++++----------- 4 files changed, 65 insertions(+), 59 deletions(-) diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 2a4f77993c..9605d61091 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -147,9 +147,9 @@ def write_to_deltalake( mode: str, max_rows_per_group: int, overwrite_schema: bool, - name: Optional[str], - description: Optional[str], - configuration: Optional[Mapping[str, Optional[str]]], + _name: Optional[str], + _description: Optional[str], + _configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index e69560e897..716f391a6e 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -38,8 +38,8 @@ from ._internal import DeltaDataChecker as _DeltaDataChecker from ._internal import batch_distinct -from ._internal import write_new_deltalake as _write_new_deltalake -from ._internal import write_to_deltalake as _write_to_deltalake +from ._internal import write_new_deltalake as write_deltalake_pyarrow +from ._internal import write_to_deltalake as write_deltalake_rust from .exceptions import DeltaProtocolError, TableNotFoundError from .table import MAX_SUPPORTED_WRITER_VERSION, DeltaTable @@ -179,6 +179,11 @@ def write_deltalake( if table is not None and mode == "ignore": return + if mode == "overwrite" and overwrite_schema: + raise NotImplementedError( + "The rust engine writer does not yet support schema evolution." + ) + if isinstance(data, RecordBatchReader): batch_iter = data elif isinstance(data, pa.RecordBatch): @@ -188,7 +193,10 @@ def write_deltalake( elif isinstance(data, ds.Dataset): batch_iter = data.scanner().to_reader() elif isinstance(data, pd.DataFrame): - batch_iter = pa.Table.from_pandas(data).to_reader() + if schema is not None: + batch_iter = pa.Table.from_pandas(data, schema).to_reader() + else: + batch_iter = pa.Table.from_pandas(data).to_reader() else: batch_iter = data @@ -199,16 +207,16 @@ def write_deltalake( raise ValueError("You must provide schema if data is Iterable") data = RecordBatchReader.from_batches(schema, (batch for batch in batch_iter)) - _write_to_deltalake( + write_deltalake_rust( table_uri=table_uri, data=data, partition_by=partition_by, mode=mode, max_rows_per_group=max_rows_per_group, overwrite_schema=overwrite_schema, - name=name, - description=description, - configuration=configuration, + _name=name, + _description=description, + _configuration=configuration, storage_options=storage_options, ) if table: @@ -412,7 +420,7 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: ) if table is None: - _write_new_deltalake( + write_deltalake_pyarrow( table_uri, schema, add_actions, diff --git a/python/src/lib.rs b/python/src/lib.rs index 5ded14c6be..8832a39d6c 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -43,7 +43,6 @@ use deltalake::DeltaTableBuilder; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::{PyFrozenSet, PyType}; -use serde_json::Value; use crate::error::DeltaProtocolError; use crate::error::PythonError; @@ -1141,40 +1140,18 @@ impl From<&PyAddAction> for Add { fn write_to_deltalake( table_uri: String, data: PyArrowType, - // schema: Option>, // maybe do the schema casting on python side mode: String, max_rows_per_group: i64, overwrite_schema: bool, partition_by: Option>, - name: Option, - description: Option, - configuration: Option>>, + _name: Option, + _description: Option, + _configuration: Option>>, storage_options: Option>, ) -> PyResult<()> { let batches = data.0.map(|batch| batch.unwrap()).collect::>(); let save_mode = save_mode_from_str(&mode)?; - let mut metadata: HashMap = HashMap::new(); - - if let Some(name) = name { - metadata.insert("name".to_string(), name.into()); - } - - if let Some(description) = description { - metadata.insert("description".to_string(), description.into()); - } - - if let Some(configuration) = configuration { - metadata.insert("configuration".to_string(), json!(configuration)); - } - - // // This should be done when the table can not be loaded ... - // match save_mode { - // SaveMode::Ignore => { - // return Ok(()) - // } - // _ => () - // } let options = storage_options.clone().unwrap_or_default(); let table = rt()? .block_on(DeltaOps::try_from_uri_with_storage_options( @@ -1186,7 +1163,6 @@ fn write_to_deltalake( .write(batches) .with_save_mode(save_mode) .with_overwrite_schema(overwrite_schema) - .with_metadata(metadata) .with_write_batch_size(max_rows_per_group as usize); if let Some(partition_columns) = partition_by { @@ -1200,7 +1176,6 @@ fn write_to_deltalake( Ok(()) } -use serde_json::json; #[pyfunction] #[allow(clippy::too_many_arguments)] fn write_new_deltalake( diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index d2a2abefa6..465a2b3e77 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -145,25 +145,29 @@ def test_update_schema(existing_table: DeltaTable): assert existing_table.schema().to_pyarrow() == new_data.schema -# def test_update_schema_rust_writer(existing_table: DeltaTable): # Test fails -# new_data = pa.table({"x": pa.array([1, 2, 3])}) - -# with pytest.raises(DeltaError): -# write_deltalake( -# existing_table, -# new_data, -# mode="append", -# overwrite_schema=True, -# engine="rust", -# ) +def test_update_schema_rust_writer(existing_table: DeltaTable): # Test fails + new_data = pa.table({"x": pa.array([1, 2, 3])}) -# write_deltalake( -# existing_table, new_data, mode="overwrite", overwrite_schema=True, engine="rust" -# ) + with pytest.raises(DeltaError): + write_deltalake( + existing_table, + new_data, + mode="append", + overwrite_schema=True, + engine="rust", + ) + with pytest.raises(NotImplementedError): + write_deltalake( + existing_table, + new_data, + mode="overwrite", + overwrite_schema=True, + engine="rust", + ) -# read_data = existing_table.to_pyarrow_table() -# assert new_data == read_data -# assert existing_table.schema().to_pyarrow() == new_data.schema + read_data = existing_table.to_pyarrow_table() + assert new_data == read_data + assert existing_table.schema().to_pyarrow() == new_data.schema @pytest.mark.parametrize("engine", ["pyarrow", "rust"]) @@ -185,15 +189,34 @@ def test_local_path( assert table == sample_data -@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) -def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table, engine): +@pytest.mark.skip(reason="Waiting on support with create matadata during write") +def test_roundtrip_metadata_rust(tmp_path: pathlib.Path, sample_data: pa.Table, engine): write_deltalake( tmp_path, sample_data, name="test_name", description="test_desc", configuration={"delta.appendOnly": "false", "foo": "bar"}, - engine=engine, + engine="rust", + ) + + delta_table = DeltaTable(tmp_path) + + metadata = delta_table.metadata() + + assert metadata.name == "test_name" + assert metadata.description == "test_desc" + assert metadata.configuration == {"delta.appendOnly": "false", "foo": "bar"} + + +def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): + write_deltalake( + tmp_path, + sample_data, + name="test_name", + description="test_desc", + configuration={"delta.appendOnly": "false", "foo": "bar"}, + engine="pyarrow", ) delta_table = DeltaTable(tmp_path) From 717b7c700266628c3e63602a0fbe3ae5a6cb6f93 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 13:07:48 +0100 Subject: [PATCH 07/35] add float type support in partition cols --- crates/deltalake-core/src/operations/mod.rs | 8 +++++--- crates/deltalake-core/src/operations/write.rs | 4 +++- crates/deltalake-core/src/writer/utils.rs | 10 ++++++---- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index 5eef40698f..fb0f25d379 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -7,13 +7,13 @@ //! with a [data stream][datafusion::physical_plan::SendableRecordBatchStream], //! if the operation returns data as well. -use std::collections::{HashMap}; use self::create::CreateBuilder; use self::filesystem_check::FileSystemCheckBuilder; use self::vacuum::VacuumBuilder; use crate::errors::{DeltaResult, DeltaTableError}; use crate::table::builder::DeltaTableBuilder; use crate::DeltaTable; +use std::collections::HashMap; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod convert_to_delta; @@ -73,10 +73,12 @@ impl DeltaOps { Err(err) => Err(err), } } - /// try from uri with storage options - pub async fn try_from_uri_with_storage_options(uri: impl AsRef, storage_options: HashMap) -> DeltaResult { + pub async fn try_from_uri_with_storage_options( + uri: impl AsRef, + storage_options: HashMap, + ) -> DeltaResult { let mut table = DeltaTableBuilder::from_uri(uri) .with_storage_options(storage_options) .build()?; diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index fdc6f82faa..f38b44fc69 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -362,7 +362,9 @@ impl std::future::IntoFuture for WriteBuilder { .or_else(|_| this.snapshot.arrow_schema()) .unwrap_or(schema.clone()); - if !can_cast_batch(schema.fields(), table_schema.fields()) && !this.overwrite_schema { + if !can_cast_batch(schema.fields(), table_schema.fields()) + && !this.overwrite_schema + { return Err(DeltaTableError::Generic( "Schema of data does not match table schema".to_string(), )); diff --git a/crates/deltalake-core/src/writer/utils.rs b/crates/deltalake-core/src/writer/utils.rs index 49c3c6bfee..5aa1ca4347 100644 --- a/crates/deltalake-core/src/writer/utils.rs +++ b/crates/deltalake-core/src/writer/utils.rs @@ -8,10 +8,10 @@ use arrow::array::{ as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array, Array, }; use arrow::datatypes::{ - DataType, Date32Type, Date64Type, Int16Type, Int32Type, Int64Type, Int8Type, - Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, TimeUnit, TimestampMicrosecondType, - TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, - UInt64Type, UInt8Type, + DataType, Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, + Int8Type, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, TimeUnit, + TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, + TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; use arrow::json::ReaderBuilder; use arrow::record_batch::*; @@ -184,6 +184,8 @@ pub(crate) fn stringified_partition_value( DataType::UInt16 => as_primitive_array::(arr).value(0).to_string(), DataType::UInt32 => as_primitive_array::(arr).value(0).to_string(), DataType::UInt64 => as_primitive_array::(arr).value(0).to_string(), + DataType::Float32 => as_primitive_array::(arr).value(0).to_string(), + DataType::Float64 => as_primitive_array::(arr).value(0).to_string(), DataType::Utf8 => as_string_array(arr).value(0).to_string(), DataType::Boolean => as_boolean_array(arr).value(0).to_string(), DataType::Date32 => as_primitive_array::(arr) From dae361ecff36d1ae1b6d428c4b048be11121459b Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 13:18:14 +0100 Subject: [PATCH 08/35] check for pandas --- python/deltalake/writer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 716f391a6e..15c7b1a5a7 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -192,7 +192,7 @@ def write_deltalake( batch_iter = data.to_reader() elif isinstance(data, ds.Dataset): batch_iter = data.scanner().to_reader() - elif isinstance(data, pd.DataFrame): + elif _has_pandas and isinstance(data, pd.DataFrame): if schema is not None: batch_iter = pa.Table.from_pandas(data, schema).to_reader() else: From 3052f121c767c4119dcebf003692d7fa90c1bae5 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 14:38:34 +0100 Subject: [PATCH 09/35] add support for name and desc --- crates/deltalake-core/src/operations/write.rs | 47 +++++++++++++++++++ python/deltalake/_internal.pyi | 4 +- python/deltalake/writer.py | 4 +- python/src/lib.rs | 12 ++++- python/tests/test_writer.py | 5 +- 5 files changed, 63 insertions(+), 9 deletions(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index f38b44fc69..440ca6dca0 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -111,6 +111,12 @@ pub struct WriteBuilder { writer_properties: Option, /// Additional metadata to be added to commit app_metadata: Option>, + /// Name of the table, only used when table doesn't exist yet + name: Option, + /// Description of the table, only used when table doesn't exist yet + description: Option, + /// Configurations of the delta table, only used when table doesn't exist + // configuration: Option>>, } impl WriteBuilder { @@ -131,6 +137,9 @@ impl WriteBuilder { overwrite_schema: false, writer_properties: None, app_metadata: None, + name: None, + description: None, + // configuration: None, } } @@ -214,6 +223,31 @@ impl WriteBuilder { self } + /// Specify the table name. Optionally qualified with + /// a database name [database_name.] table_name. + pub fn with_table_name(mut self, name: impl Into) -> Self { + self.name = Some(name.into()); + self + } + + /// Comment to describe the table. + pub fn with_description(mut self, description: impl Into) -> Self { + self.description = Some(description.into()); + self + } + + // /// Set configuration on created table + // pub fn with_configuration( + // mut self, + // configuration: impl IntoIterator, Option>)>, + // ) -> Self { + // self.configuration = configuration + // .into_iter() + // .map(|(k, v)| (k.into(), v.map(|s| s.into()))) + // .collect(); + // self + // } + async fn check_preconditions(&self) -> DeltaResult> { match self.log_store.is_delta_table_location().await? { true => { @@ -242,6 +276,19 @@ impl WriteBuilder { if let Some(partition_columns) = self.partition_columns.as_ref() { builder = builder.with_partition_columns(partition_columns.clone()) } + + if let Some(name) = self.name.as_ref() { + builder = builder.with_table_name(name.clone()); + }; + + if let Some(desc) = self.description.as_ref() { + builder = builder.with_comment(desc.clone()); + }; + + // if let Some(config) = self.configuration.as_ref() { + // builder = builder.with_configuration(config.clone()); + // }; + let (_, actions, _) = builder.into_table_and_actions()?; Ok(actions) } diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 9605d61091..1989e39e8c 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -147,8 +147,8 @@ def write_to_deltalake( mode: str, max_rows_per_group: int, overwrite_schema: bool, - _name: Optional[str], - _description: Optional[str], + name: Optional[str], + description: Optional[str], _configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 15c7b1a5a7..35323f9a49 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -214,8 +214,8 @@ def write_deltalake( mode=mode, max_rows_per_group=max_rows_per_group, overwrite_schema=overwrite_schema, - _name=name, - _description=description, + name=name, + description=description, _configuration=configuration, storage_options=storage_options, ) diff --git a/python/src/lib.rs b/python/src/lib.rs index 8832a39d6c..c22fb408f0 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1144,8 +1144,8 @@ fn write_to_deltalake( max_rows_per_group: i64, overwrite_schema: bool, partition_by: Option>, - _name: Option, - _description: Option, + name: Option, + description: Option, _configuration: Option>>, storage_options: Option>, ) -> PyResult<()> { @@ -1169,6 +1169,14 @@ fn write_to_deltalake( builder = builder.with_partition_columns(partition_columns); } + if let Some(name) = &name { + builder = builder.with_table_name(name); + }; + + if let Some(description) = &description { + builder = builder.with_description(description); + }; + rt()? .block_on(builder.into_future()) .map_err(PythonError::from)?; diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 465a2b3e77..797b9365a9 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -189,8 +189,7 @@ def test_local_path( assert table == sample_data -@pytest.mark.skip(reason="Waiting on support with create matadata during write") -def test_roundtrip_metadata_rust(tmp_path: pathlib.Path, sample_data: pa.Table, engine): +def test_roundtrip_metadata_rust(tmp_path: pathlib.Path, sample_data: pa.Table): write_deltalake( tmp_path, sample_data, @@ -206,7 +205,7 @@ def test_roundtrip_metadata_rust(tmp_path: pathlib.Path, sample_data: pa.Table, assert metadata.name == "test_name" assert metadata.description == "test_desc" - assert metadata.configuration == {"delta.appendOnly": "false", "foo": "bar"} + # assert metadata.configuration == {"delta.appendOnly": "false", "foo": "bar"} def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): From 01f01940de7f585d7587f2093407d4c57b184cfa Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 14:47:34 +0100 Subject: [PATCH 10/35] fmt --- crates/deltalake-core/src/operations/write.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index 440ca6dca0..2ef9afa4af 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -115,7 +115,7 @@ pub struct WriteBuilder { name: Option, /// Description of the table, only used when table doesn't exist yet description: Option, - /// Configurations of the delta table, only used when table doesn't exist + // /// Configurations of the delta table, only used when table doesn't exist // configuration: Option>>, } From 99115743a278d84af84dbe5f009c2f1f67279c07 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 15:38:31 +0100 Subject: [PATCH 11/35] improve tests and add config support --- crates/deltalake-core/src/operations/write.rs | 35 +++--- python/deltalake/_internal.pyi | 2 +- python/deltalake/writer.py | 27 ++--- python/src/lib.rs | 6 +- python/tests/test_writer.py | 112 +++++++----------- 5 files changed, 81 insertions(+), 101 deletions(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index 2ef9afa4af..5230fd09dd 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -115,8 +115,8 @@ pub struct WriteBuilder { name: Option, /// Description of the table, only used when table doesn't exist yet description: Option, - // /// Configurations of the delta table, only used when table doesn't exist - // configuration: Option>>, + /// Configurations of the delta table, only used when table doesn't exist + configuration: HashMap>, } impl WriteBuilder { @@ -139,7 +139,7 @@ impl WriteBuilder { app_metadata: None, name: None, description: None, - // configuration: None, + configuration: Default::default(), } } @@ -236,17 +236,17 @@ impl WriteBuilder { self } - // /// Set configuration on created table - // pub fn with_configuration( - // mut self, - // configuration: impl IntoIterator, Option>)>, - // ) -> Self { - // self.configuration = configuration - // .into_iter() - // .map(|(k, v)| (k.into(), v.map(|s| s.into()))) - // .collect(); - // self - // } + /// Set configuration on created table + pub fn with_configuration( + mut self, + configuration: impl IntoIterator, Option>)>, + ) -> Self { + self.configuration = configuration + .into_iter() + .map(|(k, v)| (k.into(), v.map(|s| s.into()))) + .collect(); + self + } async fn check_preconditions(&self) -> DeltaResult> { match self.log_store.is_delta_table_location().await? { @@ -272,7 +272,8 @@ impl WriteBuilder { }?; let mut builder = CreateBuilder::new() .with_log_store(self.log_store.clone()) - .with_columns(schema.fields().clone()); + .with_columns(schema.fields().clone()) + .with_configuration(self.configuration.clone()); if let Some(partition_columns) = self.partition_columns.as_ref() { builder = builder.with_partition_columns(partition_columns.clone()) } @@ -285,10 +286,6 @@ impl WriteBuilder { builder = builder.with_comment(desc.clone()); }; - // if let Some(config) = self.configuration.as_ref() { - // builder = builder.with_configuration(config.clone()); - // }; - let (_, actions, _) = builder.into_table_and_actions()?; Ok(actions) } diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 1989e39e8c..2a4f77993c 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -149,7 +149,7 @@ def write_to_deltalake( overwrite_schema: bool, name: Optional[str], description: Optional[str], - _configuration: Optional[Mapping[str, Optional[str]]], + configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 35323f9a49..b1e301b39e 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -125,7 +125,7 @@ def write_deltalake( table_or_uri: URI of a table or a DeltaTable object. data: Data to write. If passing iterable, the schema must also be given. schema: Optional schema to write. - partition_by: List of columns to partition the table by. Only required + partition_by: List of columns to partition the table by. Only required when creating a new table. filesystem: Optional filesystem to pass to PyArrow. If not provided will be inferred from uri. The file system has to be rooted in the table root. @@ -137,20 +137,20 @@ def write_deltalake( file_options: Optional write options for Parquet (ParquetFileWriteOptions). Can be provided with defaults using ParquetFileWriteOptions().make_write_options(). Please refer to https://github.com/apache/arrow/blob/master/python/pyarrow/_dataset_parquet.pyx#L492-L533 - for the list of available options - max_partitions: the maximum number of partitions that will be used. + for the list of available options. Only used in pyarrow engine. + max_partitions: the maximum number of partitions that will be used. Only used in pyarrow engine. max_open_files: Limits the maximum number of files that can be left open while writing. If an attempt is made to open too many files then the least recently used file will be closed. If this setting is set too low you may end up fragmenting your - data into many small files. + data into many small files. Only used in pyarrow engine. max_rows_per_file: Maximum number of rows per file. If greater than 0 then this will limit how many rows are placed in any single file. Otherwise there will be no limit and one file will be created in each output directory unless files need to be closed to respect max_open_files min_rows_per_group: Minimum number of rows per group. When the value is set, the dataset writer will batch incoming data and only write the row groups to the disk - when sufficient rows have accumulated. + when sufficient rows have accumulated. Only used in pyarrow engine. max_rows_per_group: Maximum number of rows per group. If the value is set, then the dataset writer may split up large incoming batches into multiple row groups. If this value is set, then min_rows_per_group should also be set. @@ -159,7 +159,7 @@ def write_deltalake( configuration: A map containing configuration options for the metadata action. overwrite_schema: If True, allows updating the schema of the table. storage_options: options passed to the native delta filesystem. Unused if 'filesystem' is defined. - partition_filters: the partition filters that will be used for partition overwrite. + partition_filters: the partition filters that will be used for partition overwrite. Only used in pyarrow engine. large_dtypes: If True, the table schema is checked against large_dtypes """ table, table_uri = try_get_table_and_table_uri(table_or_uri, storage_options) @@ -174,7 +174,12 @@ def write_deltalake( if table: table.update_incremental() - + if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: + raise DeltaProtocolError( + "This table's min_writer_version is " + f"{table.protocol().min_writer_version}, " + "but this method only supports version 2." + ) if engine == "rust": if table is not None and mode == "ignore": return @@ -216,7 +221,7 @@ def write_deltalake( overwrite_schema=overwrite_schema, name=name, description=description, - _configuration=configuration, + configuration=configuration, storage_options=storage_options, ) if table: @@ -264,12 +269,6 @@ def write_deltalake( else: partition_by = table.metadata().partition_columns - if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: - raise DeltaProtocolError( - "This table's min_writer_version is " - f"{table.protocol().min_writer_version}, " - "but this method only supports version 2." - ) else: # creating a new table current_version = -1 diff --git a/python/src/lib.rs b/python/src/lib.rs index c22fb408f0..eddd742951 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1146,7 +1146,7 @@ fn write_to_deltalake( partition_by: Option>, name: Option, description: Option, - _configuration: Option>>, + configuration: Option>>, storage_options: Option>, ) -> PyResult<()> { let batches = data.0.map(|batch| batch.unwrap()).collect::>(); @@ -1177,6 +1177,10 @@ fn write_to_deltalake( builder = builder.with_description(description); }; + if let Some(config) = configuration { + builder = builder.with_configuration(config); + }; + rt()? .block_on(builder.into_future()) .map_err(PythonError::from)?; diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 797b9365a9..808a2f9563 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -145,7 +145,7 @@ def test_update_schema(existing_table: DeltaTable): assert existing_table.schema().to_pyarrow() == new_data.schema -def test_update_schema_rust_writer(existing_table: DeltaTable): # Test fails +def test_update_schema_rust_writer(existing_table: DeltaTable): new_data = pa.table({"x": pa.array([1, 2, 3])}) with pytest.raises(DeltaError): @@ -156,6 +156,7 @@ def test_update_schema_rust_writer(existing_table: DeltaTable): # Test fails overwrite_schema=True, engine="rust", ) + # TODO(ion): Remove this once we add schema overwrite support with pytest.raises(NotImplementedError): write_deltalake( existing_table, @@ -189,33 +190,15 @@ def test_local_path( assert table == sample_data -def test_roundtrip_metadata_rust(tmp_path: pathlib.Path, sample_data: pa.Table): - write_deltalake( - tmp_path, - sample_data, - name="test_name", - description="test_desc", - configuration={"delta.appendOnly": "false", "foo": "bar"}, - engine="rust", - ) - - delta_table = DeltaTable(tmp_path) - - metadata = delta_table.metadata() - - assert metadata.name == "test_name" - assert metadata.description == "test_desc" - # assert metadata.configuration == {"delta.appendOnly": "false", "foo": "bar"} - - -def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table, engine): write_deltalake( tmp_path, sample_data, name="test_name", description="test_desc", configuration={"delta.appendOnly": "false", "foo": "bar"}, - engine="pyarrow", + engine=engine, ) delta_table = DeltaTable(tmp_path) @@ -300,30 +283,17 @@ def test_roundtrip_multi_partitioned( assert add_path.count("/") == 2 -def test_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): - write_deltalake(tmp_path, sample_data) - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data - - with pytest.raises(AssertionError): - write_deltalake(tmp_path, sample_data, mode="error") - - write_deltalake(tmp_path, sample_data, mode="ignore") - assert ("0" * 19 + "1.json") not in os.listdir(tmp_path / "_delta_log") - - write_deltalake(tmp_path, sample_data, mode="append") - expected = pa.concat_tables([sample_data, sample_data]) - assert DeltaTable(tmp_path).to_pyarrow_table() == expected - - write_deltalake(tmp_path, sample_data, mode="overwrite") - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data - - -def test_write_modes_rust(tmp_path: pathlib.Path, sample_data: pa.Table): - write_deltalake(tmp_path, sample_data) +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table, engine): + write_deltalake(tmp_path, sample_data, engine=engine) assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data - with pytest.raises(DeltaError): - write_deltalake(tmp_path, sample_data, mode="error", engine="rust") + if engine == "pyarrow": + with pytest.raises(AssertionError): + write_deltalake(tmp_path, sample_data, mode="error") + elif engine == "rust": + with pytest.raises(DeltaError): + write_deltalake(tmp_path, sample_data, mode="error", engine="rust") write_deltalake(tmp_path, sample_data, mode="ignore", engine="rust") assert ("0" * 19 + "1.json") not in os.listdir(tmp_path / "_delta_log") @@ -336,15 +306,18 @@ def test_write_modes_rust(tmp_path: pathlib.Path, sample_data: pa.Table): assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_append_only_should_append_only_with_the_overwrite_mode( # Create rust equivalent rust - tmp_path: pathlib.Path, sample_data: pa.Table + tmp_path: pathlib.Path, sample_data: pa.Table, engine ): config = {"delta.appendOnly": "true"} - write_deltalake(tmp_path, sample_data, mode="append", configuration=config) + write_deltalake( + tmp_path, sample_data, mode="append", configuration=config, engine=engine + ) table = DeltaTable(tmp_path) - write_deltalake(table, sample_data, mode="append") + write_deltalake(table, sample_data, mode="append", engine=engine) data_store_types = [tmp_path, table] fail_modes = ["overwrite", "ignore", "error"] @@ -357,7 +330,7 @@ def test_append_only_should_append_only_with_the_overwrite_mode( # Create rust f" 'append'. Mode is currently {mode}" ), ): - write_deltalake(data_store_type, sample_data, mode=mode) + write_deltalake(data_store_type, sample_data, mode=mode, engine=engine) expected = pa.concat_tables([sample_data, sample_data]) @@ -371,24 +344,31 @@ def test_writer_with_table(existing_table: DeltaTable, sample_data: pa.Table, en assert existing_table.to_pyarrow_table() == sample_data -def test_fails_wrong_partitioning(existing_table: DeltaTable, sample_data: pa.Table): - with pytest.raises(AssertionError): - write_deltalake( - existing_table, sample_data, mode="append", partition_by="int32" - ) - - -def test_fails_wrong_partitioning_rust_writer( - existing_table: DeltaTable, sample_data: pa.Table +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_fails_wrong_partitioning( + existing_table: DeltaTable, sample_data: pa.Table, engine ): - with pytest.raises(DeltaError): - write_deltalake( - existing_table, - sample_data, - mode="append", - partition_by="int32", - engine="rust", - ) + if engine == "pyarrow": + with pytest.raises(AssertionError): + write_deltalake( + existing_table, + sample_data, + mode="append", + partition_by="int32", + engine=engine, + ) + elif engine == "rust": + with pytest.raises( + DeltaError, + match='Generic error: Specified table partitioning does not match table partitioning: expected: [], got: ["int32"]', + ): + write_deltalake( + existing_table, + sample_data, + mode="append", + partition_by="int32", + engine=engine, + ) @pytest.mark.parametrize("engine", ["pyarrow", "rust"]) @@ -556,7 +536,7 @@ def test_writer_null_stats(tmp_path: pathlib.Path, engine: Literal["pyarrow", "r assert stats["nullCount"] == expected_nulls -@pytest.mark.parametrize("engine", ["pyarrow"]) # This one is broken +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_writer_fails_on_protocol( existing_table: DeltaTable, sample_data: pa.Table, From a410dfae0de6a447bb0aa9f0118f391fd322778d Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 15:42:52 +0100 Subject: [PATCH 12/35] parametrize write benchmark --- python/deltalake/writer.py | 2 +- python/tests/test_benchmark.py | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index b1e301b39e..01b7382751 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -125,7 +125,7 @@ def write_deltalake( table_or_uri: URI of a table or a DeltaTable object. data: Data to write. If passing iterable, the schema must also be given. schema: Optional schema to write. - partition_by: List of columns to partition the table by. Only required + partition_by: List of columns to partition the table by. Only required when creating a new table. filesystem: Optional filesystem to pass to PyArrow. If not provided will be inferred from uri. The file system has to be rooted in the table root. diff --git a/python/tests/test_benchmark.py b/python/tests/test_benchmark.py index fd32a7e4e6..d7299ca684 100644 --- a/python/tests/test_benchmark.py +++ b/python/tests/test_benchmark.py @@ -24,9 +24,12 @@ def sample_table() -> pa.Table: return tab +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) @pytest.mark.benchmark(group="write") -def test_benchmark_write(benchmark, sample_table, tmp_path): - benchmark(write_deltalake, str(tmp_path), sample_table, mode="overwrite") +def test_benchmark_write(benchmark, sample_table, tmp_path, engine): + benchmark( + write_deltalake, str(tmp_path), sample_table, mode="overwrite", engine=engine + ) dt = DeltaTable(str(tmp_path)) assert dt.to_pyarrow_table().sort_by("i") == sample_table From 8c976b68f3a40fff08e9f2ae23b4e8af7829386a Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 18 Nov 2023 17:34:31 +0100 Subject: [PATCH 13/35] add LargeUtf8 support in partition stringify --- crates/deltalake-core/src/writer/utils.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/deltalake-core/src/writer/utils.rs b/crates/deltalake-core/src/writer/utils.rs index 5aa1ca4347..173340f368 100644 --- a/crates/deltalake-core/src/writer/utils.rs +++ b/crates/deltalake-core/src/writer/utils.rs @@ -5,7 +5,8 @@ use std::io::Write; use std::sync::Arc; use arrow::array::{ - as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array, Array, + as_boolean_array, as_generic_binary_array, as_largestring_array, as_primitive_array, + as_string_array, Array, }; use arrow::datatypes::{ DataType, Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, @@ -187,6 +188,7 @@ pub(crate) fn stringified_partition_value( DataType::Float32 => as_primitive_array::(arr).value(0).to_string(), DataType::Float64 => as_primitive_array::(arr).value(0).to_string(), DataType::Utf8 => as_string_array(arr).value(0).to_string(), + DataType::LargeUtf8 => as_largestring_array(arr).value(0).to_string(), DataType::Boolean => as_boolean_array(arr).value(0).to_string(), DataType::Date32 => as_primitive_array::(arr) .value_as_date(0) From 57565b57db0e05ae79524d6e5d1bbc437e614015 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sat, 18 Nov 2023 12:25:42 +0100 Subject: [PATCH 14/35] refactor: express log schema in delta types --- .../deltalake-core/src/kernel/actions/mod.rs | 10 +- .../src/kernel/actions/schemas.rs | 481 ++++---- .../src/kernel/actions/types.rs | 125 +- .../kernel/{actions/arrow.rs => arrow/mod.rs} | 17 +- .../src/kernel/arrow/schemas.rs | 63 + crates/deltalake-core/src/kernel/error.rs | 3 +- crates/deltalake-core/src/kernel/mod.rs | 2 + .../src/protocol/checkpoints.rs | 2 +- .../src/schema/arrow_convert.rs | 1049 ----------------- crates/deltalake-core/src/table/config.rs | 56 + 10 files changed, 466 insertions(+), 1342 deletions(-) rename crates/deltalake-core/src/kernel/{actions/arrow.rs => arrow/mod.rs} (98%) create mode 100644 crates/deltalake-core/src/kernel/arrow/schemas.rs delete mode 100644 crates/deltalake-core/src/schema/arrow_convert.rs diff --git a/crates/deltalake-core/src/kernel/actions/mod.rs b/crates/deltalake-core/src/kernel/actions/mod.rs index 865c9d3cd9..637d520c41 100644 --- a/crates/deltalake-core/src/kernel/actions/mod.rs +++ b/crates/deltalake-core/src/kernel/actions/mod.rs @@ -7,9 +7,7 @@ use std::collections::HashMap; use serde::{Deserialize, Serialize}; -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub(crate) mod arrow; -// pub(crate) mod schemas; +pub(crate) mod schemas; mod serde_path; pub(crate) mod types; @@ -32,10 +30,12 @@ pub enum ActionType { Protocol, /// modify the data in a table by removing individual logical files Remove, - /// The Row ID high-water mark tracks the largest ID that has been assigned to a row in the table. - RowIdHighWaterMark, /// Transactional information Txn, + /// Checkpoint metadata + CheckpointMetadata, + /// Sidecar + Sidecar, } #[derive(Debug, PartialEq, Eq, Clone, Deserialize, Serialize)] diff --git a/crates/deltalake-core/src/kernel/actions/schemas.rs b/crates/deltalake-core/src/kernel/actions/schemas.rs index 0cc870318f..ad3e3ccbad 100644 --- a/crates/deltalake-core/src/kernel/actions/schemas.rs +++ b/crates/deltalake-core/src/kernel/actions/schemas.rs @@ -1,255 +1,262 @@ -use std::sync::Arc; +//! Schema definitions for action types -use arrow_schema::{DataType, Field, Fields, Schema}; +use lazy_static::lazy_static; use super::ActionType; +use crate::kernel::schema::{ArrayType, DataType, MapType, StructField, StructType}; -impl ActionType { - /// Returns the root field for the action type - pub fn field(&self) -> Field { - match self { - Self::Add => get_root("add", self.fields()), - Self::Cdc => get_root("cdc", self.fields()), - Self::CommitInfo => get_root("commitInfo", self.fields()), - Self::DomainMetadata => get_root("domainMetadata", self.fields()), - Self::Metadata => get_root("metaData", self.fields()), - Self::Protocol => get_root("protocol", self.fields()), - Self::Remove => get_root("remove", self.fields()), - Self::RowIdHighWaterMark => get_root("rowIdHighWaterMark", self.fields()), - Self::Txn => get_root("txn", self.fields()), - } - } - - /// Returns the child fields for the action type - pub fn fields(&self) -> Vec { - match self { - Self::Add => add_fields(), - Self::Cdc => cdc_fields(), - Self::CommitInfo => commit_info_fields(), - Self::DomainMetadata => domain_metadata_fields(), - Self::Metadata => metadata_fields(), - Self::Protocol => protocol_fields(), - Self::Remove => remove_fields(), - Self::RowIdHighWaterMark => watermark_fields(), - Self::Txn => txn_fields(), - } - } -} - -/// Returns the schema for the delta log -pub fn get_log_schema() -> Schema { - Schema { - fields: Fields::from_iter([ - ActionType::Add.field(), - ActionType::Cdc.field(), - ActionType::CommitInfo.field(), - ActionType::DomainMetadata.field(), - ActionType::Metadata.field(), - ActionType::Protocol.field(), - ActionType::Remove.field(), - ActionType::RowIdHighWaterMark.field(), - ActionType::Txn.field(), - ]), - metadata: Default::default(), - } -} - -fn get_root(name: &str, fields: Vec) -> Field { - Field::new(name, DataType::Struct(Fields::from_iter(fields)), true) -} - -fn add_fields() -> Vec { - Vec::from_iter([ - Field::new("path", DataType::Utf8, false), - Field::new("size", DataType::Int64, false), - Field::new("modificationTime", DataType::Int64, false), - Field::new("dataChange", DataType::Boolean, false), - Field::new("stats", DataType::Utf8, true), - Field::new( - "partitionValues", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "tags", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "deletionVector", - DataType::Struct(Fields::from(vec![ - Field::new("storageType", DataType::Utf8, false), - Field::new("pathOrInlineDv", DataType::Utf8, false), - Field::new("offset", DataType::Int32, true), - Field::new("sizeInBytes", DataType::Int32, false), - Field::new("cardinality", DataType::Int64, false), - ])), - true, - ), - Field::new("baseRowId", DataType::Int64, true), - Field::new("defaultRowCommitVersion", DataType::Int64, true), - ]) -} - -fn cdc_fields() -> Vec { - Vec::from_iter([ - Field::new("path", DataType::Utf8, true), - Field::new( - "partitionValues", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new("size", DataType::Int64, true), - Field::new("dataChange", DataType::Boolean, true), - Field::new( - "tags", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - ]) -} - -fn remove_fields() -> Vec { - Vec::from_iter([ - Field::new("path", DataType::Utf8, true), - Field::new("deletionTimestamp", DataType::Int64, true), - Field::new("dataChange", DataType::Boolean, true), - Field::new("extendedFileMetadata", DataType::Boolean, true), - Field::new("size", DataType::Int64, true), - Field::new( - "partitionValues", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "tags", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - ]) -} - -fn metadata_fields() -> Vec { - Vec::from_iter([ - Field::new("id", DataType::Utf8, false), - Field::new("name", DataType::Utf8, true), - Field::new("description", DataType::Utf8, true), - Field::new( - "format", - DataType::Struct(Fields::from_iter([ - Field::new("provider", DataType::Utf8, true), - Field::new( - "options", - DataType::Map( - Arc::new(Field::new( - "key_value", - DataType::Struct(Fields::from_iter([ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Utf8, true), - ])), - false, - )), - false, +lazy_static! { + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#change-metadata + static ref METADATA_FIELD: StructField = StructField::new( + "metaData", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("id", DataType::string(), false), + StructField::new("name", DataType::string(), true), + StructField::new("description", DataType::string(), true), + StructField::new( + "format", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("provider", DataType::string(), false), + StructField::new( + "configuration", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + true, ), - false, - ), - ])), - false, - ), - Field::new("schemaString", DataType::Utf8, false), - Field::new("createdTime", DataType::Int64, true), - Field::new( - "partitionColumns", - DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), - false, - ), - Field::new( - "configuration", - DataType::Map( - Arc::new(Field::new( - "key_value", - DataType::Struct(Fields::from_iter([ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Utf8, true), - ])), - false, - )), + ]))), false, ), - true, - ), - ]) + StructField::new("schemaString", DataType::string(), false), + StructField::new( + "partitionColumns", + DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), + false, + ), + StructField::new("createdTime", DataType::long(), true), + StructField::new( + "configuration", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + false, + ), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#protocol-evolution + static ref PROTOCOL_FIELD: StructField = StructField::new( + "protocol", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("minReaderVersion", DataType::integer(), false), + StructField::new("minWriterVersion", DataType::integer(), false), + StructField::new( + "readerFeatures", + DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), + true, + ), + StructField::new( + "writerFeatures", + DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), + true, + ), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#commit-provenance-information + static ref COMMIT_INFO_FIELD: StructField = StructField::new( + "commitInfo", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("timestamp", DataType::timestamp(), false), + StructField::new("operation", DataType::string(), false), + StructField::new("isolationLevel", DataType::string(), true), + StructField::new("isBlindAppend", DataType::boolean(), true), + StructField::new("txnId", DataType::string(), true), + StructField::new("readVersion", DataType::long(), true), + StructField::new( + "operationParameters", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + true, + ), + StructField::new( + "operationMetrics", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + true, + ), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file + static ref ADD_FIELD: StructField = StructField::new( + "add", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + partition_values_field(), + StructField::new("size", DataType::long(), false), + StructField::new("modificationTime", DataType::timestamp(), false), + StructField::new("dataChange", DataType::boolean(), false), + StructField::new("stats", DataType::string(), true), + tags_field(), + deletion_vector_field(), + StructField::new("baseRowId", DataType::long(), true), + StructField::new("defaultRowCommitVersion", DataType::long(), true), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file + static ref REMOVE_FIELD: StructField = StructField::new( + "remove", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + StructField::new("deletionTimestamp", DataType::timestamp(), true), + StructField::new("dataChange", DataType::boolean(), false), + StructField::new("extendedFileMetadata", DataType::boolean(), true), + partition_values_field(), + StructField::new("size", DataType::long(), true), + StructField::new("stats", DataType::string(), true), + tags_field(), + deletion_vector_field(), + StructField::new("baseRowId", DataType::long(), true), + StructField::new("defaultRowCommitVersion", DataType::long(), true), + ]))), + true, + ); + static ref REMOVE_FIELD_CHECKPOINT: StructField = StructField::new( + "remove", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + StructField::new("deletionTimestamp", DataType::timestamp(), true), + StructField::new("dataChange", DataType::boolean(), false), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-cdc-file + static ref CDC_FIELD: StructField = StructField::new( + "cdc", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + partition_values_field(), + StructField::new("size", DataType::long(), false), + StructField::new("dataChange", DataType::boolean(), false), + tags_field(), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#transaction-identifiers + static ref TXN_FIELD: StructField = StructField::new( + "txn", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("appId", DataType::string(), false), + StructField::new("version", DataType::long(), false), + StructField::new("lastUpdated", DataType::timestamp(), true), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#domain-metadata + static ref DOMAIN_METADATA_FIELD: StructField = StructField::new( + "domainMetadata", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("domain", DataType::string(), false), + StructField::new( + "configuration", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + false, + ), + StructField::new("removed", DataType::boolean(), false), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#checkpoint-metadata + static ref CHECKPOINT_METADATA_FIELD: StructField = StructField::new( + "checkpointMetadata", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("flavor", DataType::string(), false), + tags_field(), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#sidecar-file-information + static ref SIDECAR_FIELD: StructField = StructField::new( + "sidecar", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + StructField::new("sizeInBytes", DataType::long(), false), + StructField::new("modificationTime", DataType::timestamp(), false), + StructField::new("type", DataType::string(), false), + tags_field(), + ]))), + true, + ); } -fn protocol_fields() -> Vec { - Vec::from_iter([ - Field::new("minReaderVersion", DataType::Int32, false), - Field::new("minWriterVersion", DataType::Int32, false), - Field::new( - "readerFeatures", - DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), +fn tags_field() -> StructField { + StructField::new( + "tags", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), true, - ), - Field::new( - "writerFeatures", - DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), - true, - ), - ]) -} - -fn txn_fields() -> Vec { - Vec::from_iter([ - Field::new("appId", DataType::Utf8, true), - Field::new("version", DataType::Int64, true), - Field::new("lastUpdated", DataType::Int64, true), - ]) -} - -fn watermark_fields() -> Vec { - Vec::from_iter([Field::new("highWaterMark", DataType::Int64, true)]) + ))), + true, + ) } -fn commit_info_fields() -> Vec { - Vec::from_iter([ - Field::new("timestamp", DataType::Int64, true), - Field::new("operation", DataType::Utf8, true), - Field::new("isolationLevel", DataType::Utf8, true), - Field::new("isBlindAppend", DataType::Boolean, true), - Field::new("txnId", DataType::Utf8, true), - Field::new("readVersion", DataType::Int32, true), - Field::new( - "operationParameters", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "operationMetrics", - DataType::Map(Arc::new(get_map_field()), false), +fn partition_values_field() -> StructField { + StructField::new( + "partitionValues", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), true, - ), - ]) + ))), + false, + ) } -fn domain_metadata_fields() -> Vec { - Vec::from_iter([ - Field::new("domain", DataType::Utf8, true), - Field::new( - "configuration", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new("removed", DataType::Boolean, true), - ]) +fn deletion_vector_field() -> StructField { + StructField::new( + "deletionVector", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("storageType", DataType::string(), false), + StructField::new("pathOrInlineDv", DataType::string(), false), + StructField::new("offset", DataType::integer(), true), + StructField::new("sizeInBytes", DataType::integer(), false), + StructField::new("cardinality", DataType::long(), false), + ]))), + true, + ) } -fn get_map_field() -> Field { - Field::new( - "key_value", - DataType::Struct(Fields::from_iter([ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Utf8, true), - ])), - false, - ) +impl ActionType { + /// Returns the type of the corresponding field in the delta log schema + pub fn schema_field(&self) -> &StructField { + match self { + Self::Metadata => &METADATA_FIELD, + Self::Protocol => &PROTOCOL_FIELD, + Self::CommitInfo => &COMMIT_INFO_FIELD, + Self::Add => &ADD_FIELD, + Self::Remove => &REMOVE_FIELD, + Self::Cdc => &CDC_FIELD, + Self::Txn => &TXN_FIELD, + Self::DomainMetadata => &DOMAIN_METADATA_FIELD, + Self::CheckpointMetadata => &CHECKPOINT_METADATA_FIELD, + Self::Sidecar => &SIDECAR_FIELD, + } + } } diff --git a/crates/deltalake-core/src/kernel/actions/types.rs b/crates/deltalake-core/src/kernel/actions/types.rs index a788315b82..aa60823e4a 100644 --- a/crates/deltalake-core/src/kernel/actions/types.rs +++ b/crates/deltalake-core/src/kernel/actions/types.rs @@ -174,7 +174,7 @@ pub enum ReaderFeatures { /// Mapping of one column to another ColumnMapping, /// Deletion vectors for merge, update, delete - DeleteionVecotrs, + DeletionVectors, /// timestamps without timezone support #[serde(alias = "timestampNtz")] TimestampWithoutTimezone, @@ -185,26 +185,13 @@ pub enum ReaderFeatures { Other(String), } -#[allow(clippy::from_over_into)] -impl Into for ReaderFeatures { - fn into(self) -> usize { - match self { - ReaderFeatures::Other(_) => 0, - ReaderFeatures::ColumnMapping => 2, - ReaderFeatures::DeleteionVecotrs - | ReaderFeatures::TimestampWithoutTimezone - | ReaderFeatures::V2Checkpoint => 3, - } - } -} - #[cfg(all(not(feature = "parquet2"), feature = "parquet"))] impl From<&parquet::record::Field> for ReaderFeatures { fn from(value: &parquet::record::Field) -> Self { match value { parquet::record::Field::Str(feature) => match feature.as_str() { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeleteionVecotrs, + "deletionVectors" => ReaderFeatures::DeletionVectors, "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -216,9 +203,15 @@ impl From<&parquet::record::Field> for ReaderFeatures { impl From for ReaderFeatures { fn from(value: String) -> Self { - match value.as_str() { + value.as_str().into() + } +} + +impl From<&str> for ReaderFeatures { + fn from(value: &str) -> Self { + match value { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeleteionVecotrs, + "deletionVectors" => ReaderFeatures::DeletionVectors, "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -230,7 +223,7 @@ impl AsRef for ReaderFeatures { fn as_ref(&self) -> &str { match self { ReaderFeatures::ColumnMapping => "columnMapping", - ReaderFeatures::DeleteionVecotrs => "deletionVectors", + ReaderFeatures::DeletionVectors => "deletionVectors", ReaderFeatures::TimestampWithoutTimezone => "timestampNtz", ReaderFeatures::V2Checkpoint => "v2Checkpoint", ReaderFeatures::Other(f) => f, @@ -264,7 +257,7 @@ pub enum WriterFeatures { /// ID Columns IdentityColumns, /// Deletion vectors for merge, update, delete - DeleteionVecotrs, + DeletionVectors, /// Row tracking on tables RowTracking, /// timestamps without timezone support @@ -281,29 +274,15 @@ pub enum WriterFeatures { Other(String), } -#[allow(clippy::from_over_into)] -impl Into for WriterFeatures { - fn into(self) -> usize { - match self { - WriterFeatures::Other(_) => 0, - WriterFeatures::AppendOnly | WriterFeatures::Invariants => 2, - WriterFeatures::CheckConstraints => 3, - WriterFeatures::ChangeDataFeed | WriterFeatures::GeneratedColumns => 4, - WriterFeatures::ColumnMapping => 5, - WriterFeatures::IdentityColumns - | WriterFeatures::DeleteionVecotrs - | WriterFeatures::RowTracking - | WriterFeatures::TimestampWithoutTimezone - | WriterFeatures::DomainMetadata - | WriterFeatures::V2Checkpoint - | WriterFeatures::IcebergCompatV1 => 7, - } +impl From for WriterFeatures { + fn from(value: String) -> Self { + value.as_str().into() } } -impl From for WriterFeatures { - fn from(value: String) -> Self { - match value.as_str() { +impl From<&str> for WriterFeatures { + fn from(value: &str) -> Self { + match value { "appendOnly" => WriterFeatures::AppendOnly, "invariants" => WriterFeatures::Invariants, "checkConstraints" => WriterFeatures::CheckConstraints, @@ -311,7 +290,7 @@ impl From for WriterFeatures { "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeleteionVecotrs, + "deletionVectors" => WriterFeatures::DeletionVectors, "rowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, @@ -332,7 +311,7 @@ impl AsRef for WriterFeatures { WriterFeatures::GeneratedColumns => "generatedColumns", WriterFeatures::ColumnMapping => "columnMapping", WriterFeatures::IdentityColumns => "identityColumns", - WriterFeatures::DeleteionVecotrs => "deletionVectors", + WriterFeatures::DeletionVectors => "deletionVectors", WriterFeatures::RowTracking => "rowTracking", WriterFeatures::TimestampWithoutTimezone => "timestampNtz", WriterFeatures::DomainMetadata => "domainMetadata", @@ -361,7 +340,7 @@ impl From<&parquet::record::Field> for WriterFeatures { "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeleteionVecotrs, + "deletionVectors" => WriterFeatures::DeletionVectors, "rowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, @@ -421,7 +400,7 @@ impl AsRef for StorageType { impl ToString for StorageType { fn to_string(&self) -> String { - self.as_ref().to_string() + self.as_ref().into() } } @@ -450,6 +429,7 @@ pub struct DeletionVectorDescriptor { /// Start of the data for this DV in number of bytes from the beginning of the file it is stored in. /// Always None (absent in JSON) when `storageType = 'i'`. + #[serde(skip_serializing_if = "Option::is_none")] pub offset: Option, /// Size of the serialized DV in bytes (raw data size, i.e. before base85 encoding, if inline). @@ -662,9 +642,11 @@ pub struct Remove { pub data_change: bool, /// The time this logical file was created, as milliseconds since the epoch. + #[serde(skip_serializing_if = "Option::is_none")] pub deletion_timestamp: Option, /// When true the fields `partition_values`, `size`, and `tags` are present + #[serde(skip_serializing_if = "Option::is_none")] pub extended_file_metadata: Option, /// A map from partition column to value for this logical file. @@ -686,9 +668,11 @@ pub struct Remove { /// Default generated Row ID of the first row in the file. The default generated Row IDs /// of the other rows in the file can be reconstructed by adding the physical index of the /// row within the file to the base Row ID + #[serde(skip_serializing_if = "Option::is_none")] pub base_row_id: Option, /// First commit version in which an add action with the same path was committed to the table. + #[serde(skip_serializing_if = "Option::is_none")] pub default_row_commit_version: Option, } @@ -707,13 +691,18 @@ pub struct AddCDCFile { /// absolute path to a CDC file #[serde(with = "serde_path")] pub path: String, + /// The size of this file in bytes pub size: i64, + /// A map from partition column to value for this file pub partition_values: HashMap>, + /// Should always be set to false because they do not change the underlying data of the table pub data_change: bool, + /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, } @@ -724,9 +713,12 @@ pub struct AddCDCFile { pub struct Txn { /// A unique identifier for the application performing the transaction. pub app_id: String, + /// An application-specific numeric identifier for this transaction. pub version: i64, + /// The time when this transaction action was created in milliseconds since the Unix epoch. + #[serde(skip_serializing_if = "Option::is_none")] pub last_updated: Option, } @@ -739,30 +731,39 @@ pub struct CommitInfo { /// Timestamp in millis when the commit was created #[serde(skip_serializing_if = "Option::is_none")] pub timestamp: Option, + /// Id of the user invoking the commit #[serde(skip_serializing_if = "Option::is_none")] pub user_id: Option, + /// Name of the user invoking the commit #[serde(skip_serializing_if = "Option::is_none")] pub user_name: Option, + /// The operation performed during the #[serde(skip_serializing_if = "Option::is_none")] pub operation: Option, + /// Parameters used for table operation #[serde(skip_serializing_if = "Option::is_none")] pub operation_parameters: Option>, + /// Version of the table when the operation was started #[serde(skip_serializing_if = "Option::is_none")] pub read_version: Option, + /// The isolation level of the commit #[serde(skip_serializing_if = "Option::is_none")] pub isolation_level: Option, + /// TODO #[serde(skip_serializing_if = "Option::is_none")] pub is_blind_append: Option, + /// Delta engine which created the commit. #[serde(skip_serializing_if = "Option::is_none")] pub engine_info: Option, + /// Additional provenance information for the commit #[serde(flatten, default)] pub info: HashMap, @@ -774,12 +775,50 @@ pub struct CommitInfo { pub struct DomainMetadata { /// Identifier for this domain (system or user-provided) pub domain: String, + /// String containing configuration for the metadata domain pub configuration: String, + /// When `true` the action serves as a tombstone pub removed: bool, } +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +/// This action is only allowed in checkpoints following V2 spec. It describes the details about the checkpoint. +pub struct CheckpointMetadata { + /// The flavor of the V2 checkpoint. Allowed values: "flat". + pub flavor: String, + + /// Map containing any additional metadata about the v2 spec checkpoint. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + +/// The sidecar action references a sidecar file which provides some of the checkpoint's file actions. +/// This action is only allowed in checkpoints following V2 spec. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct Sidecar { + /// The name of the sidecar file (not a path). + /// The file must reside in the _delta_log/_sidecars directory. + pub file_name: String, + + /// The size of the sidecar file in bytes + pub size_in_bytes: i64, + + /// The time this sidecar file was created, as milliseconds since the epoch. + pub modification_time: i64, + + /// Type of sidecar. Valid values are: "fileaction". + /// This could be extended in future to allow different kinds of sidecars. + #[serde(rename = "type")] + pub sidecar_type: String, + + /// Map containing any additional metadata about the checkpoint sidecar file. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + #[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] /// The isolation level applied during transaction pub enum IsolationLevel { diff --git a/crates/deltalake-core/src/kernel/actions/arrow.rs b/crates/deltalake-core/src/kernel/arrow/mod.rs similarity index 98% rename from crates/deltalake-core/src/kernel/actions/arrow.rs rename to crates/deltalake-core/src/kernel/arrow/mod.rs index d292362604..0c89f6ab48 100644 --- a/crates/deltalake-core/src/kernel/actions/arrow.rs +++ b/crates/deltalake-core/src/kernel/arrow/mod.rs @@ -1,3 +1,5 @@ +//! Conversions between Delta and Arrow data types + use std::sync::Arc; use arrow_schema::{ @@ -6,7 +8,12 @@ use arrow_schema::{ }; use lazy_static::lazy_static; -use super::super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; +use super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; + +pub mod schemas; + +const MAP_KEYS_NAME: &str = "keys"; +const MAP_VALUES_NAME: &str = "values"; impl TryFrom<&StructType> for ArrowSchema { type Error = ArrowError; @@ -64,9 +71,9 @@ impl TryFrom<&MapType> for ArrowField { "entries", ArrowDataType::Struct( vec![ - ArrowField::new("key", ArrowDataType::try_from(a.key_type())?, false), + ArrowField::new(MAP_KEYS_NAME, ArrowDataType::try_from(a.key_type())?, false), ArrowField::new( - "value", + MAP_VALUES_NAME, ArrowDataType::try_from(a.value_type())?, a.value_contains_null(), ), @@ -143,12 +150,12 @@ impl TryFrom<&DataType> for ArrowDataType { ArrowDataType::Struct( vec![ ArrowField::new( - "keys", + MAP_KEYS_NAME, >::try_from(m.key_type())?, false, ), ArrowField::new( - "values", + MAP_VALUES_NAME, >::try_from(m.value_type())?, m.value_contains_null(), ), diff --git a/crates/deltalake-core/src/kernel/arrow/schemas.rs b/crates/deltalake-core/src/kernel/arrow/schemas.rs new file mode 100644 index 0000000000..80a29e065e --- /dev/null +++ b/crates/deltalake-core/src/kernel/arrow/schemas.rs @@ -0,0 +1,63 @@ +//! Arrow schemas for the delta log + +use arrow_schema::{Field, Fields, Schema}; +use lazy_static::lazy_static; + +use super::super::ActionType; + +lazy_static! { + static ref ARROW_METADATA_FIELD: Field = + ActionType::Metadata.schema_field().try_into().unwrap(); + static ref ARROW_PROTOCOL_FIELD: Field = + ActionType::Protocol.schema_field().try_into().unwrap(); + static ref ARROW_COMMIT_INFO_FIELD: Field = + ActionType::CommitInfo.schema_field().try_into().unwrap(); + static ref ARROW_ADD_FIELD: Field = ActionType::Add.schema_field().try_into().unwrap(); + static ref ARROW_REMOVE_FIELD: Field = ActionType::Remove.schema_field().try_into().unwrap(); + static ref ARROW_CDC_FIELD: Field = ActionType::Cdc.schema_field().try_into().unwrap(); + static ref ARROW_TXN_FIELD: Field = ActionType::Txn.schema_field().try_into().unwrap(); + static ref ARROW_DOMAIN_METADATA_FIELD: Field = ActionType::DomainMetadata + .schema_field() + .try_into() + .unwrap(); + static ref ARROW_CHECKPOINT_METADATA_FIELD: Field = ActionType::CheckpointMetadata + .schema_field() + .try_into() + .unwrap(); + static ref ARROW_SIDECAR_FIELD: Field = ActionType::Sidecar.schema_field().try_into().unwrap(); +} + +impl ActionType { + /// Returns the root field for the action type + pub fn arrow_field(&self) -> &Field { + match self { + Self::Metadata => &ARROW_METADATA_FIELD, + Self::Protocol => &ARROW_PROTOCOL_FIELD, + Self::CommitInfo => &ARROW_COMMIT_INFO_FIELD, + Self::Add => &ARROW_ADD_FIELD, + Self::Remove => &ARROW_REMOVE_FIELD, + Self::Cdc => &ARROW_CDC_FIELD, + Self::Txn => &ARROW_TXN_FIELD, + Self::DomainMetadata => &ARROW_DOMAIN_METADATA_FIELD, + Self::CheckpointMetadata => &ARROW_CHECKPOINT_METADATA_FIELD, + Self::Sidecar => &ARROW_SIDECAR_FIELD, + } + } +} + +/// Returns the schema for the delta log +pub fn get_log_schema() -> Schema { + Schema { + fields: Fields::from_iter([ + ActionType::Add.arrow_field().clone(), + ActionType::Cdc.arrow_field().clone(), + ActionType::CommitInfo.arrow_field().clone(), + ActionType::DomainMetadata.arrow_field().clone(), + ActionType::Metadata.arrow_field().clone(), + ActionType::Protocol.arrow_field().clone(), + ActionType::Remove.arrow_field().clone(), + ActionType::Txn.arrow_field().clone(), + ]), + metadata: Default::default(), + } +} diff --git a/crates/deltalake-core/src/kernel/error.rs b/crates/deltalake-core/src/kernel/error.rs index 8ec799ca96..a37dbdae67 100644 --- a/crates/deltalake-core/src/kernel/error.rs +++ b/crates/deltalake-core/src/kernel/error.rs @@ -23,9 +23,8 @@ pub enum Error { #[error("Arrow error: {0}")] Parquet(#[from] parquet::errors::ParquetError), - #[cfg(feature = "object_store")] #[error("Error interacting with object store: {0}")] - ObjectStore(object_store::Error), + ObjectStore(#[from] object_store::Error), #[error("File not found: {0}")] FileNotFound(String), diff --git a/crates/deltalake-core/src/kernel/mod.rs b/crates/deltalake-core/src/kernel/mod.rs index 7785c273f9..54f742c3fb 100644 --- a/crates/deltalake-core/src/kernel/mod.rs +++ b/crates/deltalake-core/src/kernel/mod.rs @@ -1,6 +1,8 @@ //! Kernel module pub mod actions; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod arrow; pub mod error; pub mod schema; diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs index a4cc1b66c7..837483c35c 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -18,7 +18,7 @@ use regex::Regex; use serde_json::Value; use super::{time_utils, ProtocolError}; -use crate::kernel::actions::arrow::delta_log_schema_for_table; +use crate::kernel::arrow::delta_log_schema_for_table; use crate::kernel::{ Action, Add as AddAction, DataType, Metadata, PrimitiveType, Protocol, StructField, StructType, Txn, diff --git a/crates/deltalake-core/src/schema/arrow_convert.rs b/crates/deltalake-core/src/schema/arrow_convert.rs deleted file mode 100644 index d292362604..0000000000 --- a/crates/deltalake-core/src/schema/arrow_convert.rs +++ /dev/null @@ -1,1049 +0,0 @@ -use std::sync::Arc; - -use arrow_schema::{ - ArrowError, DataType as ArrowDataType, Field as ArrowField, FieldRef as ArrowFieldRef, - Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, TimeUnit, -}; -use lazy_static::lazy_static; - -use super::super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; - -impl TryFrom<&StructType> for ArrowSchema { - type Error = ArrowError; - - fn try_from(s: &StructType) -> Result { - let fields = s - .fields() - .iter() - .map(>::try_from) - .collect::, ArrowError>>()?; - - Ok(ArrowSchema::new(fields)) - } -} - -impl TryFrom<&StructField> for ArrowField { - type Error = ArrowError; - - fn try_from(f: &StructField) -> Result { - let metadata = f - .metadata() - .iter() - .map(|(key, val)| Ok((key.clone(), serde_json::to_string(val)?))) - .collect::>() - .map_err(|err| ArrowError::JsonError(err.to_string()))?; - - let field = ArrowField::new( - f.name(), - ArrowDataType::try_from(f.data_type())?, - f.is_nullable(), - ) - .with_metadata(metadata); - - Ok(field) - } -} - -impl TryFrom<&ArrayType> for ArrowField { - type Error = ArrowError; - - fn try_from(a: &ArrayType) -> Result { - Ok(ArrowField::new( - "item", - ArrowDataType::try_from(a.element_type())?, - a.contains_null(), - )) - } -} - -impl TryFrom<&MapType> for ArrowField { - type Error = ArrowError; - - fn try_from(a: &MapType) -> Result { - Ok(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::try_from(a.key_type())?, false), - ArrowField::new( - "value", - ArrowDataType::try_from(a.value_type())?, - a.value_contains_null(), - ), - ] - .into(), - ), - false, // always non-null - )) - } -} - -impl TryFrom<&DataType> for ArrowDataType { - type Error = ArrowError; - - fn try_from(t: &DataType) -> Result { - match t { - DataType::Primitive(p) => { - match p { - PrimitiveType::String => Ok(ArrowDataType::Utf8), - PrimitiveType::Long => Ok(ArrowDataType::Int64), // undocumented type - PrimitiveType::Integer => Ok(ArrowDataType::Int32), - PrimitiveType::Short => Ok(ArrowDataType::Int16), - PrimitiveType::Byte => Ok(ArrowDataType::Int8), - PrimitiveType::Float => Ok(ArrowDataType::Float32), - PrimitiveType::Double => Ok(ArrowDataType::Float64), - PrimitiveType::Boolean => Ok(ArrowDataType::Boolean), - PrimitiveType::Binary => Ok(ArrowDataType::Binary), - PrimitiveType::Decimal(precision, scale) => { - let precision = u8::try_from(*precision).map_err(|_| { - ArrowError::SchemaError(format!( - "Invalid precision for decimal: {}", - precision - )) - })?; - let scale = i8::try_from(*scale).map_err(|_| { - ArrowError::SchemaError(format!("Invalid scale for decimal: {}", scale)) - })?; - - if precision <= 38 { - Ok(ArrowDataType::Decimal128(precision, scale)) - } else if precision <= 76 { - Ok(ArrowDataType::Decimal256(precision, scale)) - } else { - Err(ArrowError::SchemaError(format!( - "Precision too large to be represented in Arrow: {}", - precision - ))) - } - } - PrimitiveType::Date => { - // A calendar date, represented as a year-month-day triple without a - // timezone. Stored as 4 bytes integer representing days since 1970-01-01 - Ok(ArrowDataType::Date32) - } - PrimitiveType::Timestamp => { - // Issue: https://github.com/delta-io/delta/issues/643 - Ok(ArrowDataType::Timestamp(TimeUnit::Microsecond, None)) - } - } - } - DataType::Struct(s) => Ok(ArrowDataType::Struct( - s.fields() - .iter() - .map(>::try_from) - .collect::, ArrowError>>()? - .into(), - )), - DataType::Array(a) => Ok(ArrowDataType::List(Arc::new(>::try_from(a)?))), - DataType::Map(m) => Ok(ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new( - "keys", - >::try_from(m.key_type())?, - false, - ), - ArrowField::new( - "values", - >::try_from(m.value_type())?, - m.value_contains_null(), - ), - ] - .into(), - ), - false, - )), - false, - )), - } - } -} - -impl TryFrom<&ArrowSchema> for StructType { - type Error = ArrowError; - - fn try_from(arrow_schema: &ArrowSchema) -> Result { - let new_fields: Result, _> = arrow_schema - .fields() - .iter() - .map(|field| field.as_ref().try_into()) - .collect(); - Ok(StructType::new(new_fields?)) - } -} - -impl TryFrom for StructType { - type Error = ArrowError; - - fn try_from(arrow_schema: ArrowSchemaRef) -> Result { - arrow_schema.as_ref().try_into() - } -} - -impl TryFrom<&ArrowField> for StructField { - type Error = ArrowError; - - fn try_from(arrow_field: &ArrowField) -> Result { - Ok(StructField::new( - arrow_field.name().clone(), - arrow_field.data_type().try_into()?, - arrow_field.is_nullable(), - ) - .with_metadata(arrow_field.metadata().iter().map(|(k, v)| (k.clone(), v)))) - } -} - -impl TryFrom<&ArrowDataType> for DataType { - type Error = ArrowError; - - fn try_from(arrow_datatype: &ArrowDataType) -> Result { - match arrow_datatype { - ArrowDataType::Utf8 => Ok(DataType::Primitive(PrimitiveType::String)), - ArrowDataType::LargeUtf8 => Ok(DataType::Primitive(PrimitiveType::String)), - ArrowDataType::Int64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type - ArrowDataType::Int32 => Ok(DataType::Primitive(PrimitiveType::Integer)), - ArrowDataType::Int16 => Ok(DataType::Primitive(PrimitiveType::Short)), - ArrowDataType::Int8 => Ok(DataType::Primitive(PrimitiveType::Byte)), - ArrowDataType::UInt64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type - ArrowDataType::UInt32 => Ok(DataType::Primitive(PrimitiveType::Integer)), - ArrowDataType::UInt16 => Ok(DataType::Primitive(PrimitiveType::Short)), - ArrowDataType::UInt8 => Ok(DataType::Primitive(PrimitiveType::Boolean)), - ArrowDataType::Float32 => Ok(DataType::Primitive(PrimitiveType::Float)), - ArrowDataType::Float64 => Ok(DataType::Primitive(PrimitiveType::Double)), - ArrowDataType::Boolean => Ok(DataType::Primitive(PrimitiveType::Boolean)), - ArrowDataType::Binary => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::FixedSizeBinary(_) => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::LargeBinary => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::Decimal128(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( - *p as i32, *s as i32, - ))), - ArrowDataType::Decimal256(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( - *p as i32, *s as i32, - ))), - ArrowDataType::Date32 => Ok(DataType::Primitive(PrimitiveType::Date)), - ArrowDataType::Date64 => Ok(DataType::Primitive(PrimitiveType::Date)), - ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => { - Ok(DataType::Primitive(PrimitiveType::Timestamp)) - } - ArrowDataType::Timestamp(TimeUnit::Microsecond, Some(tz)) - if tz.eq_ignore_ascii_case("utc") => - { - Ok(DataType::Primitive(PrimitiveType::Timestamp)) - } - ArrowDataType::Struct(fields) => { - let converted_fields: Result, _> = fields - .iter() - .map(|field| field.as_ref().try_into()) - .collect(); - Ok(DataType::Struct(Box::new(StructType::new( - converted_fields?, - )))) - } - ArrowDataType::List(field) => Ok(DataType::Array(Box::new(ArrayType::new( - (*field).data_type().try_into()?, - (*field).is_nullable(), - )))), - ArrowDataType::LargeList(field) => Ok(DataType::Array(Box::new(ArrayType::new( - (*field).data_type().try_into()?, - (*field).is_nullable(), - )))), - ArrowDataType::FixedSizeList(field, _) => Ok(DataType::Array(Box::new( - ArrayType::new((*field).data_type().try_into()?, (*field).is_nullable()), - ))), - ArrowDataType::Map(field, _) => { - if let ArrowDataType::Struct(struct_fields) = field.data_type() { - let key_type = struct_fields[0].data_type().try_into()?; - let value_type = struct_fields[1].data_type().try_into()?; - let value_type_nullable = struct_fields[1].is_nullable(); - Ok(DataType::Map(Box::new(MapType::new( - key_type, - value_type, - value_type_nullable, - )))) - } else { - panic!("DataType::Map should contain a struct field child"); - } - } - s => Err(ArrowError::SchemaError(format!( - "Invalid data type for Delta Lake: {s}" - ))), - } - } -} - -macro_rules! arrow_map { - ($fieldname: ident, null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Utf8, false), - ArrowField::new("value", ArrowDataType::Utf8, true), - ] - .into(), - ), - false, - )), - false, - ), - true, - ) - }; - ($fieldname: ident, not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Utf8, false), - ArrowField::new("value", ArrowDataType::Utf8, false), - ] - .into(), - ), - false, - )), - false, - ), - false, - ) - }; -} - -macro_rules! arrow_field { - ($fieldname:ident, $type_qual:ident, null) => { - ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, true) - }; - ($fieldname:ident, $type_qual:ident, not_null) => { - ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, false) - }; -} - -macro_rules! arrow_list { - ($fieldname:ident, $element_name:ident, $type_qual:ident, null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::List(Arc::new(ArrowField::new( - stringify!($element_name), - ArrowDataType::$type_qual, - true, - ))), - true, - ) - }; - ($fieldname:ident, $element_name:ident, $type_qual:ident, not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::List(Arc::new(ArrowField::new( - stringify!($element_name), - ArrowDataType::$type_qual, - true, - ))), - false, - ) - }; -} - -macro_rules! arrow_struct { - ($fieldname:ident, [$($inner:tt)+], null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Struct( - arrow_defs! [$($inner)+].into() - ), - true - ) - }; - ($fieldname:ident, [$($inner:tt)+], not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Struct( - arrow_defs! [$($inner)+].into() - ), - false - ) - } -} - -macro_rules! arrow_def { - ($fieldname:ident $(null)?) => { - arrow_map!($fieldname, null) - }; - ($fieldname:ident not_null) => { - arrow_map!($fieldname, not_null) - }; - ($fieldname:ident[$inner_name:ident]{$type_qual:ident} $(null)?) => { - arrow_list!($fieldname, $inner_name, $type_qual, null) - }; - ($fieldname:ident[$inner_name:ident]{$type_qual:ident} not_null) => { - arrow_list!($fieldname, $inner_name, $type_qual, not_null) - }; - ($fieldname:ident:$type_qual:ident $(null)?) => { - arrow_field!($fieldname, $type_qual, null) - }; - ($fieldname:ident:$type_qual:ident not_null) => { - arrow_field!($fieldname, $type_qual, not_null) - }; - ($fieldname:ident[$($inner:tt)+] $(null)?) => { - arrow_struct!($fieldname, [$($inner)+], null) - }; - ($fieldname:ident[$($inner:tt)+] not_null) => { - arrow_struct!($fieldname, [$($inner)+], not_null) - } -} - -/// A helper macro to create more readable Arrow field definitions, delimited by commas -/// -/// The argument patterns are as follows: -/// -/// fieldname (null|not_null)? -- An arrow field of type map with name "fieldname" consisting of Utf8 key-value pairs, and an -/// optional nullability qualifier (null if not specified). -/// -/// fieldname:type (null|not_null)? -- An Arrow field consisting of an atomic type. For example, -/// id:Utf8 gets mapped to ArrowField::new("id", ArrowDataType::Utf8, true). -/// where customerCount:Int64 not_null gets mapped to gets mapped to -/// ArrowField::new("customerCount", ArrowDataType::Utf8, true) -/// -/// fieldname[list_element]{list_element_type} (null|not_null)? -- An Arrow list, with the name of the elements wrapped in square brackets -/// and the type of the list elements wrapped in curly brackets. For example, -/// customers[name]{Utf8} is an nullable arrow field of type arrow list consisting -/// of elements called "name" with type Utf8. -/// -/// fieldname[element1, element2, element3, ....] (null|not_null)? -- An arrow struct with name "fieldname" consisting of elements adhering to any of the patterns -/// documented, including additional structs arbitrarily nested up to the recursion -/// limit for Rust macros. -macro_rules! arrow_defs { - () => { - vec![] as Vec - }; - ($($fieldname:ident$(:$type_qual:ident)?$([$($inner:tt)+])?$({$list_type_qual:ident})? $($nullable:ident)?),+) => { - vec![ - $(arrow_def!($fieldname$(:$type_qual)?$([$($inner)+])?$({$list_type_qual})? $($nullable)?)),+ - ] - } -} - -/// Returns an arrow schema representing the delta log for use in checkpoints -/// -/// # Arguments -/// -/// * `table_schema` - The arrow schema representing the table backed by the delta log -/// * `partition_columns` - The list of partition columns of the table. -/// * `use_extended_remove_schema` - Whether to include extended file metadata in remove action schema. -/// Required for compatibility with different versions of Databricks runtime. -pub(crate) fn delta_log_schema_for_table( - table_schema: ArrowSchema, - partition_columns: &[String], - use_extended_remove_schema: bool, -) -> ArrowSchemaRef { - lazy_static! { - static ref SCHEMA_FIELDS: Vec = arrow_defs![ - metaData[ - id:Utf8, - name:Utf8, - description:Utf8, - schemaString:Utf8, - createdTime:Int64, - partitionColumns[element]{Utf8}, - configuration, - format[provider:Utf8, options] - ], - protocol[ - minReaderVersion:Int32, - minWriterVersion:Int32 - ], - txn[ - appId:Utf8, - version:Int64 - ] - ]; - static ref ADD_FIELDS: Vec = arrow_defs![ - path:Utf8, - size:Int64, - modificationTime:Int64, - dataChange:Boolean, - stats:Utf8, - partitionValues, - tags, - deletionVector[ - storageType:Utf8 not_null, - pathOrInlineDv:Utf8 not_null, - offset:Int32 null, - sizeInBytes:Int32 not_null, - cardinality:Int64 not_null - ] - ]; - static ref REMOVE_FIELDS: Vec = arrow_defs![ - path: Utf8, - deletionTimestamp: Int64, - dataChange: Boolean, - extendedFileMetadata: Boolean - ]; - static ref REMOVE_EXTENDED_FILE_METADATA_FIELDS: Vec = - arrow_defs![size: Int64, partitionValues, tags]; - }; - - // create add fields according to the specific data table schema - let (partition_fields, non_partition_fields): (Vec, Vec) = - table_schema - .fields() - .iter() - .map(|field| field.to_owned()) - .partition(|field| partition_columns.contains(field.name())); - - let mut stats_parsed_fields: Vec = - vec![ArrowField::new("numRecords", ArrowDataType::Int64, true)]; - if !non_partition_fields.is_empty() { - let mut max_min_vec = Vec::new(); - non_partition_fields - .iter() - .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); - - stats_parsed_fields.extend(["minValues", "maxValues"].into_iter().map(|name| { - ArrowField::new( - name, - ArrowDataType::Struct(max_min_vec.clone().into()), - true, - ) - })); - - let mut null_count_vec = Vec::new(); - non_partition_fields - .iter() - .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); - let null_count_struct = ArrowField::new( - "nullCount", - ArrowDataType::Struct(null_count_vec.into()), - true, - ); - - stats_parsed_fields.push(null_count_struct); - } - let mut add_fields = ADD_FIELDS.clone(); - add_fields.push(ArrowField::new( - "stats_parsed", - ArrowDataType::Struct(stats_parsed_fields.into()), - true, - )); - if !partition_fields.is_empty() { - add_fields.push(ArrowField::new( - "partitionValues_parsed", - ArrowDataType::Struct(partition_fields.into()), - true, - )); - } - - // create remove fields with or without extendedFileMetadata - let mut remove_fields = REMOVE_FIELDS.clone(); - if use_extended_remove_schema { - remove_fields.extend(REMOVE_EXTENDED_FILE_METADATA_FIELDS.clone()); - } - - // include add and remove fields in checkpoint schema - let mut schema_fields = SCHEMA_FIELDS.clone(); - schema_fields.push(ArrowField::new( - "add", - ArrowDataType::Struct(add_fields.into()), - true, - )); - schema_fields.push(ArrowField::new( - "remove", - ArrowDataType::Struct(remove_fields.into()), - true, - )); - - let arrow_schema = ArrowSchema::new(schema_fields); - - std::sync::Arc::new(arrow_schema) -} - -fn max_min_schema_for_fields(dest: &mut Vec, f: &ArrowField) { - match f.data_type() { - ArrowDataType::Struct(struct_fields) => { - let mut child_dest = Vec::new(); - - for f in struct_fields { - max_min_schema_for_fields(&mut child_dest, f); - } - - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); - } - // don't compute min or max for list, map or binary types - ArrowDataType::List(_) | ArrowDataType::Map(_, _) | ArrowDataType::Binary => { /* noop */ } - _ => { - let f = f.clone(); - dest.push(f); - } - } -} - -fn null_count_schema_for_fields(dest: &mut Vec, f: &ArrowField) { - match f.data_type() { - ArrowDataType::Struct(struct_fields) => { - let mut child_dest = Vec::new(); - - for f in struct_fields { - null_count_schema_for_fields(&mut child_dest, f); - } - - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); - } - _ => { - let f = ArrowField::new(f.name(), ArrowDataType::Int64, true); - dest.push(f); - } - } -} - -#[cfg(test)] -mod tests { - use arrow::array::ArrayData; - use arrow_array::Array; - use arrow_array::{make_array, ArrayRef, MapArray, StringArray, StructArray}; - use arrow_buffer::{Buffer, ToByteSlice}; - use arrow_schema::Field; - - use super::*; - use std::collections::HashMap; - use std::sync::Arc; - - #[test] - fn delta_log_schema_for_table_test() { - // NOTE: We should future proof the checkpoint schema in case action schema changes. - // See https://github.com/delta-io/delta-rs/issues/287 - - let table_schema = ArrowSchema::new(vec![ - ArrowField::new("pcol", ArrowDataType::Int32, true), - ArrowField::new("col1", ArrowDataType::Int32, true), - ]); - let partition_columns = vec!["pcol".to_string()]; - let log_schema = - delta_log_schema_for_table(table_schema.clone(), partition_columns.as_slice(), false); - - // verify top-level schema contains all expected fields and they are named correctly. - let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; - for f in log_schema.fields().iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - assert_eq!(5, log_schema.fields().len()); - - // verify add fields match as expected. a lot of transformation goes into these. - let add_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "add") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); - assert_eq!( - vec![ - "path", - "size", - "modificationTime", - "dataChange", - "stats", - "partitionValues", - "tags", - "deletionVector", - "stats_parsed", - "partitionValues_parsed" - ], - field_names - ); - let add_field_map: HashMap<_, _> = add_fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - let partition_values_parsed = add_field_map.get("partitionValues_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = partition_values_parsed.data_type() { - assert_eq!(1, fields.len()); - let field = fields.get(0).unwrap().to_owned(); - assert_eq!( - Arc::new(ArrowField::new("pcol", ArrowDataType::Int32, true)), - field - ); - } else { - unreachable!(); - } - let stats_parsed = add_field_map.get("stats_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = stats_parsed.data_type() { - assert_eq!(4, fields.len()); - - let field_map: HashMap<_, _> = fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - - for (k, v) in field_map.iter() { - match k.as_ref() { - "minValues" | "maxValues" | "nullCount" => match v.data_type() { - ArrowDataType::Struct(fields) => { - assert_eq!(1, fields.len()); - let field = fields.get(0).unwrap().to_owned(); - let data_type = if k == "nullCount" { - ArrowDataType::Int64 - } else { - ArrowDataType::Int32 - }; - assert_eq!(Arc::new(ArrowField::new("col1", data_type, true)), field); - } - _ => unreachable!(), - }, - "numRecords" => {} - _ => panic!(), - } - } - } else { - unreachable!(); - } - - // verify extended remove schema fields **ARE NOT** included when `use_extended_remove_schema` is false. - let num_remove_fields = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .count(); - assert_eq!(4, num_remove_fields); - - // verify extended remove schema fields **ARE** included when `use_extended_remove_schema` is true. - let log_schema = - delta_log_schema_for_table(table_schema, partition_columns.as_slice(), true); - let remove_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - assert_eq!(7, remove_fields.len()); - let expected_fields = [ - "path", - "deletionTimestamp", - "dataChange", - "extendedFileMetadata", - "partitionValues", - "size", - "tags", - ]; - for f in remove_fields.iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - } - - #[test] - fn test_arrow_from_delta_decimal_type() { - let precision = 20; - let scale = 2; - let decimal_field = DataType::Primitive(PrimitiveType::Decimal(precision, scale)); - assert_eq!( - >::try_from(&decimal_field).unwrap(), - ArrowDataType::Decimal128(precision as u8, scale as i8) - ); - } - - #[test] - fn test_arrow_from_delta_timestamp_type() { - let timestamp_field = DataType::Primitive(PrimitiveType::Timestamp); - assert_eq!( - >::try_from(×tamp_field).unwrap(), - ArrowDataType::Timestamp(TimeUnit::Microsecond, None) - ); - } - - #[test] - fn test_delta_from_arrow_timestamp_type() { - let timestamp_field = ArrowDataType::Timestamp(TimeUnit::Microsecond, None); - assert_eq!( - >::try_from(×tamp_field).unwrap(), - DataType::Primitive(PrimitiveType::Timestamp) - ); - } - - #[test] - fn test_delta_from_arrow_timestamp_type_with_tz() { - let timestamp_field = - ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())); - assert_eq!( - >::try_from(×tamp_field).unwrap(), - DataType::Primitive(PrimitiveType::Timestamp) - ); - } - - #[test] - fn test_delta_from_arrow_map_type() { - let arrow_map = ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int8, false), - ArrowField::new("value", ArrowDataType::Binary, true), - ] - .into(), - ), - false, - )), - false, - ); - let converted_map: DataType = (&arrow_map).try_into().unwrap(); - - assert_eq!( - converted_map, - DataType::Map(Box::new(MapType::new( - DataType::Primitive(PrimitiveType::Byte), - DataType::Primitive(PrimitiveType::Binary), - true, - ))) - ); - } - - #[test] - fn test_record_batch_from_map_type() { - let keys = vec!["0", "1", "5", "6", "7"]; - let values: Vec<&[u8]> = vec![ - b"test_val_1", - b"test_val_2", - b"long_test_val_3", - b"4", - b"test_val_5", - ]; - let entry_offsets = vec![0u32, 1, 1, 4, 5, 5]; - let num_rows = keys.len(); - - // Copied the function `new_from_string` with the patched code from https://github.com/apache/arrow-rs/pull/4808 - // This should be reverted back [`MapArray::new_from_strings`] once arrow is upgraded in this project. - fn new_from_strings<'a>( - keys: impl Iterator, - values: &dyn Array, - entry_offsets: &[u32], - ) -> Result { - let entry_offsets_buffer = Buffer::from(entry_offsets.to_byte_slice()); - let keys_data = StringArray::from_iter_values(keys); - - let keys_field = Arc::new(Field::new("keys", ArrowDataType::Utf8, false)); - let values_field = Arc::new(Field::new( - "values", - values.data_type().clone(), - values.null_count() > 0, - )); - - let entry_struct = StructArray::from(vec![ - (keys_field, Arc::new(keys_data) as ArrayRef), - (values_field, make_array(values.to_data())), - ]); - - let map_data_type = ArrowDataType::Map( - Arc::new(Field::new( - "entries", - entry_struct.data_type().clone(), - false, - )), - false, - ); - - let map_data = ArrayData::builder(map_data_type) - .len(entry_offsets.len() - 1) - .add_buffer(entry_offsets_buffer) - .add_child_data(entry_struct.into_data()) - .build()?; - - Ok(MapArray::from(map_data)) - } - - let map_array = new_from_strings( - keys.into_iter(), - &arrow::array::BinaryArray::from(values), - entry_offsets.as_slice(), - ) - .expect("Could not create a map array"); - - let schema = - >::try_from(&StructType::new(vec![ - StructField::new( - "example".to_string(), - DataType::Map(Box::new(MapType::new( - DataType::Primitive(PrimitiveType::String), - DataType::Primitive(PrimitiveType::Binary), - false, - ))), - false, - ), - ])) - .expect("Could not get schema"); - - let record_batch = - arrow::record_batch::RecordBatch::try_new(Arc::new(schema), vec![Arc::new(map_array)]) - .expect("Failed to create RecordBatch"); - - assert_eq!(record_batch.num_columns(), 1); - assert_eq!(record_batch.num_rows(), num_rows); - } - - #[test] - fn test_max_min_schema_for_fields() { - let mut max_min_vec: Vec = Vec::new(); - let fields = [ - ArrowField::new("simple", ArrowDataType::Int32, true), - ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ArrowField::new("simple", ArrowDataType::Int32, true)].into(), - ), - true, - ), - ArrowField::new( - "list", - ArrowDataType::List(Arc::new(ArrowField::new( - "simple", - ArrowDataType::Int32, - true, - ))), - true, - ), - ArrowField::new( - "map", - ArrowDataType::Map( - Arc::new(ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )), - true, - ), - true, - ), - ArrowField::new("binary", ArrowDataType::Binary, true), - ]; - - let expected = vec![fields[0].clone(), fields[1].clone()]; - - fields - .iter() - .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); - - assert_eq!(max_min_vec, expected); - } - - #[test] - fn test_null_count_schema_for_fields() { - let mut null_count_vec: Vec = Vec::new(); - let fields = [ - ArrowField::new("int32", ArrowDataType::Int32, true), - ArrowField::new("int64", ArrowDataType::Int64, true), - ArrowField::new("Utf8", ArrowDataType::Utf8, true), - ArrowField::new( - "list", - ArrowDataType::List(Arc::new(ArrowField::new( - "simple", - ArrowDataType::Int32, - true, - ))), - true, - ), - ArrowField::new( - "map", - ArrowDataType::Map( - Arc::new(ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )), - true, - ), - true, - ), - ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ArrowField::new("int32", ArrowDataType::Int32, true)].into(), - ), - true, - ), - ]; - let expected = vec![ - ArrowField::new(fields[0].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[1].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[2].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[3].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[4].name(), ArrowDataType::Int64, true), - ArrowField::new( - fields[5].name(), - ArrowDataType::Struct( - vec![ArrowField::new("int32", ArrowDataType::Int64, true)].into(), - ), - true, - ), - ]; - fields - .iter() - .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); - assert_eq!(null_count_vec, expected); - } - - /* - * This test validates the trait implementation of - * TryFrom<&Arc> for schema::SchemaField which is required with Arrow 37 since - * iterators on Fields will give an &Arc - */ - #[test] - fn tryfrom_arrowfieldref_with_structs() { - let field = Arc::new(ArrowField::new( - "test_struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )); - let _converted: StructField = field.as_ref().try_into().unwrap(); - } -} diff --git a/crates/deltalake-core/src/table/config.rs b/crates/deltalake-core/src/table/config.rs index 3fa021ce6e..5b82b401b6 100644 --- a/crates/deltalake-core/src/table/config.rs +++ b/crates/deltalake-core/src/table/config.rs @@ -11,6 +11,7 @@ use crate::errors::DeltaTableError; /// /// #[derive(PartialEq, Eq, Hash)] +#[non_exhaustive] pub enum DeltaConfigKey { /// true for this Delta table to be append-only. If append-only, /// existing records cannot be deleted, and existing values cannot be updated. @@ -100,6 +101,9 @@ pub enum DeltaConfigKey { /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. TuneFileSizesForRewrites, + + /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. + CheckpointPolicy, } impl AsRef for DeltaConfigKey { @@ -111,6 +115,7 @@ impl AsRef for DeltaConfigKey { Self::AutoOptimizeOptimizeWrite => "delta.autoOptimize.optimizeWrite", Self::CheckpointWriteStatsAsJson => "delta.checkpoint.writeStatsAsJson", Self::CheckpointWriteStatsAsStruct => "delta.checkpoint.writeStatsAsStruct", + Self::CheckpointPolicy => "delta.checkpointPolicy", Self::ColumnMappingMode => "delta.columnMapping.mode", Self::DataSkippingNumIndexedCols => "delta.dataSkippingNumIndexedCols", Self::DeletedFileRetentionDuration => "delta.deletedFileRetentionDuration", @@ -140,6 +145,7 @@ impl FromStr for DeltaConfigKey { "delta.autoOptimize.optimizeWrite" => Ok(Self::AutoOptimizeOptimizeWrite), "delta.checkpoint.writeStatsAsJson" => Ok(Self::CheckpointWriteStatsAsJson), "delta.checkpoint.writeStatsAsStruct" => Ok(Self::CheckpointWriteStatsAsStruct), + "delta.checkpointPolicy" => Ok(Self::CheckpointPolicy), "delta.columnMapping.mode" => Ok(Self::ColumnMappingMode), "delta.dataSkippingNumIndexedCols" => Ok(Self::DataSkippingNumIndexedCols), "delta.deletedFileRetentionDuration" | "deletedFileRetentionDuration" => { @@ -280,6 +286,14 @@ impl<'a> TableConfig<'a> { .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) .unwrap_or_default() } + + /// Policy applied during chepoint creation + pub fn checkpoint_policy(&self) -> CheckpointPolicy { + self.0 + .get(DeltaConfigKey::CheckpointPolicy.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) + .unwrap_or_default() + } } #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] @@ -338,6 +352,48 @@ impl FromStr for IsolationLevel { } } +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] +/// The checkpoint policy applied when writing checkpoints +#[serde(rename_all = "camelCase")] +pub enum CheckpointPolicy { + /// classic Delta Lake checkpoints + Classic, + /// v2 checkpoints + V2, + /// unknown checkpoint policy + Other(String), +} + +impl Default for CheckpointPolicy { + fn default() -> Self { + Self::Classic + } +} + +impl AsRef for CheckpointPolicy { + fn as_ref(&self) -> &str { + match self { + Self::Classic => "classic", + Self::V2 => "v2", + Self::Other(s) => s, + } + } +} + +impl FromStr for CheckpointPolicy { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "classic" => Ok(Self::Classic), + "v2" => Ok(Self::V2), + _ => Err(DeltaTableError::Generic( + "Invalid string for CheckpointPolicy".into(), + )), + } + } +} + const SECONDS_PER_MINUTE: u64 = 60; const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; From 49a298b4c7d9d4e53b089b1717d9d3b0e1860a85 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Sun, 19 Nov 2023 16:11:33 +0100 Subject: [PATCH 15/35] feat(python): expose `convert_to_deltalake` (#1842) Exposes added `convert to delta` functionality by @junjunjd to Python API. - closes #1767 --------- Co-authored-by: Robert Pack <42610831+roeap@users.noreply.github.com> --- .../src/operations/convert_to_delta.rs | 16 ++- crates/deltalake-core/src/protocol/mod.rs | 20 +++- python/deltalake/__init__.py | 1 + python/deltalake/_internal.pyi | 10 ++ python/deltalake/writer.py | 55 +++++++++++ python/src/lib.rs | 68 ++++++++++--- python/tests/test_convert_to_delta.py | 97 +++++++++++++++++++ 7 files changed, 254 insertions(+), 13 deletions(-) create mode 100644 python/tests/test_convert_to_delta.py diff --git a/crates/deltalake-core/src/operations/convert_to_delta.rs b/crates/deltalake-core/src/operations/convert_to_delta.rs index 84fffa1578..644591727c 100644 --- a/crates/deltalake-core/src/operations/convert_to_delta.rs +++ b/crates/deltalake-core/src/operations/convert_to_delta.rs @@ -27,7 +27,7 @@ use serde_json::{Map, Value}; use std::{ collections::{HashMap, HashSet}, num::TryFromIntError, - str::Utf8Error, + str::{FromStr, Utf8Error}, sync::Arc, }; @@ -82,6 +82,20 @@ pub enum PartitionStrategy { Hive, } +impl FromStr for PartitionStrategy { + type Err = DeltaTableError; + + fn from_str(s: &str) -> DeltaResult { + match s.to_ascii_lowercase().as_str() { + "hive" => Ok(PartitionStrategy::Hive), + _ => Err(DeltaTableError::Generic(format!( + "Invalid partition strategy provided {}", + s + ))), + } + } +} + /// Build an operation to convert a Parquet table to a [`DeltaTable`] in place pub struct ConvertToDeltaBuilder { log_store: Option, diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index 8a5cd9f858..e2add9b529 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -23,8 +23,9 @@ use std::borrow::Borrow; use std::collections::HashMap; use std::hash::{Hash, Hasher}; use std::mem::take; +use std::str::FromStr; -use crate::errors::DeltaResult; +use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Add, CommitInfo, Metadata, Protocol, Remove}; use crate::logstore::LogStore; use crate::table::CheckPoint; @@ -589,6 +590,23 @@ pub enum SaveMode { Ignore, } +impl FromStr for SaveMode { + type Err = DeltaTableError; + + fn from_str(s: &str) -> DeltaResult { + match s.to_ascii_lowercase().as_str() { + "append" => Ok(SaveMode::Append), + "overwrite" => Ok(SaveMode::Overwrite), + "error" => Ok(SaveMode::ErrorIfExists), + "ignore" => Ok(SaveMode::Ignore), + _ => Err(DeltaTableError::Generic(format!( + "Invalid save mode provided: {}, only these are supported: ['append', 'overwrite', 'error', 'ignore']", + s + ))), + } + } +} + /// The OutputMode used in streaming operations. #[derive(Serialize, Deserialize, Debug, Clone)] pub enum OutputMode { diff --git a/python/deltalake/__init__.py b/python/deltalake/__init__.py index 129eaff1cf..b10a708309 100644 --- a/python/deltalake/__init__.py +++ b/python/deltalake/__init__.py @@ -6,4 +6,5 @@ from .schema import Schema as Schema from .table import DeltaTable as DeltaTable from .table import Metadata as Metadata +from .writer import convert_to_deltalake as convert_to_deltalake from .writer import write_deltalake as write_deltalake diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 2a4f77993c..854d96d4a8 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -152,6 +152,16 @@ def write_to_deltalake( configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... +def convert_to_deltalake( + uri: str, + partition_by: Optional[pyarrow.Schema], + partition_strategy: Optional[Literal["hive"]], + name: Optional[str], + description: Optional[str], + configuration: Optional[Mapping[str, Optional[str]]], + storage_options: Optional[Dict[str, str]], + custom_metadata: Optional[Dict[str, str]], +) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... # Can't implement inheritance (see note in src/schema.rs), so this is next diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 01b7382751..d4b55368d5 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -38,6 +38,7 @@ from ._internal import DeltaDataChecker as _DeltaDataChecker from ._internal import batch_distinct +from ._internal import convert_to_deltalake as _convert_to_deltalake from ._internal import write_new_deltalake as write_deltalake_pyarrow from ._internal import write_to_deltalake as write_deltalake_rust from .exceptions import DeltaProtocolError, TableNotFoundError @@ -443,6 +444,60 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: raise ValueError("Only `pyarrow` or `rust` are valid inputs for the engine.") +def convert_to_deltalake( + uri: Union[str, Path], + mode: Literal["error", "ignore"] = "error", + partition_by: Optional[pa.Schema] = None, + partition_strategy: Optional[Literal["hive"]] = None, + name: Optional[str] = None, + description: Optional[str] = None, + configuration: Optional[Mapping[str, Optional[str]]] = None, + storage_options: Optional[Dict[str, str]] = None, + custom_metadata: Optional[Dict[str, str]] = None, +) -> None: + """ + `Convert` parquet tables `to delta` tables. + + Currently only HIVE partitioned tables are supported. `Convert to delta` creates + a transaction log commit with add actions, and additional properties provided such + as configuration, name, and description. + + Args: + uri: URI of a table. + partition_by: Optional partitioning schema if table is partitioned. + partition_strategy: Optional partition strategy to read and convert + mode: How to handle existing data. Default is to error if table already exists. + If 'ignore', will not convert anything if table already exists. + name: User-provided identifier for this table. + description: User-provided description for this table. + configuration: A map containing configuration options for the metadata action. + storage_options: options passed to the native delta filesystem. Unused if 'filesystem' is defined. + custom_metadata: custom metadata that will be added to the transaction commit + """ + if partition_by is not None and partition_strategy is None: + raise ValueError("Partition strategy has to be provided with partition_by.") + + if partition_strategy is not None and partition_strategy != "hive": + raise ValueError( + "Currently only `hive` partition strategy is supported to be converted." + ) + + if mode == "ignore" and try_get_deltatable(uri, storage_options) is not None: + return + + _convert_to_deltalake( + str(uri), + partition_by, + partition_strategy, + name, + description, + configuration, + storage_options, + custom_metadata, + ) + return + + def __enforce_append_only( table: Optional[DeltaTable], configuration: Optional[Mapping[str, Optional[str]]], diff --git a/python/src/lib.rs b/python/src/lib.rs index eddd742951..86bbe1f16c 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -27,6 +27,7 @@ use deltalake::datafusion::prelude::SessionContext; use deltalake::delta_datafusion::DeltaDataChecker; use deltalake::errors::DeltaTableError; use deltalake::kernel::{Action, Add, Invariant, Metadata, Remove, StructType}; +use deltalake::operations::convert_to_delta::{ConvertToDeltaBuilder, PartitionStrategy}; use deltalake::operations::delete::DeleteBuilder; use deltalake::operations::filesystem_check::FileSystemCheckBuilder; use deltalake::operations::merge::MergeBuilder; @@ -43,6 +44,7 @@ use deltalake::DeltaTableBuilder; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::{PyFrozenSet, PyType}; +use serde_json::{Map, Value}; use crate::error::DeltaProtocolError; use crate::error::PythonError; @@ -758,7 +760,8 @@ impl RawDeltaTable { schema: PyArrowType, partitions_filters: Option>, ) -> PyResult<()> { - let mode = save_mode_from_str(mode)?; + let mode = mode.parse().map_err(PythonError::from)?; + let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; let existing_schema = self._table.get_schema().map_err(PythonError::from)?; @@ -1088,16 +1091,6 @@ fn batch_distinct(batch: PyArrowType) -> PyResult PyResult { - match value { - "append" => Ok(SaveMode::Append), - "overwrite" => Ok(SaveMode::Overwrite), - "error" => Ok(SaveMode::ErrorIfExists), - "ignore" => Ok(SaveMode::Ignore), - _ => Err(PyValueError::new_err("Invalid save mode")), - } -} - fn current_timestamp() -> i64 { let start = SystemTime::now(); let since_the_epoch = start @@ -1233,6 +1226,58 @@ fn write_new_deltalake( Ok(()) } +#[pyfunction] +#[allow(clippy::too_many_arguments)] +fn convert_to_deltalake( + uri: String, + partition_schema: Option>, + partition_strategy: Option, + name: Option, + description: Option, + configuration: Option>>, + storage_options: Option>, + custom_metadata: Option>, +) -> PyResult<()> { + let mut builder = ConvertToDeltaBuilder::new().with_location(uri); + + if let Some(part_schema) = partition_schema { + let schema: StructType = (&part_schema.0).try_into().map_err(PythonError::from)?; + builder = builder.with_partition_schema(schema.fields().clone()); + } + + if let Some(partition_strategy) = &partition_strategy { + let strategy: PartitionStrategy = partition_strategy.parse().map_err(PythonError::from)?; + builder = builder.with_partition_strategy(strategy); + } + + if let Some(name) = &name { + builder = builder.with_table_name(name); + } + + if let Some(description) = &description { + builder = builder.with_comment(description); + } + + if let Some(config) = configuration { + builder = builder.with_configuration(config); + }; + + if let Some(strg_options) = storage_options { + builder = builder.with_storage_options(strg_options); + }; + + if let Some(metadata) = custom_metadata { + let json_metadata: Map = + metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); + builder = builder.with_metadata(json_metadata); + }; + + rt()? + .block_on(builder.into_future()) + .map_err(PythonError::from)?; + Ok(()) +} + #[pyclass(name = "DeltaDataChecker", module = "deltalake._internal")] struct PyDeltaDataChecker { inner: DeltaDataChecker, @@ -1279,6 +1324,7 @@ fn _internal(py: Python, m: &PyModule) -> PyResult<()> { m.add_function(pyo3::wrap_pyfunction!(rust_core_version, m)?)?; m.add_function(pyo3::wrap_pyfunction!(write_new_deltalake, m)?)?; m.add_function(pyo3::wrap_pyfunction!(write_to_deltalake, m)?)?; + m.add_function(pyo3::wrap_pyfunction!(convert_to_deltalake, m)?)?; m.add_function(pyo3::wrap_pyfunction!(batch_distinct, m)?)?; m.add_class::()?; m.add_class::()?; diff --git a/python/tests/test_convert_to_delta.py b/python/tests/test_convert_to_delta.py new file mode 100644 index 0000000000..29badf3358 --- /dev/null +++ b/python/tests/test_convert_to_delta.py @@ -0,0 +1,97 @@ +import pathlib + +import pyarrow as pa +import pyarrow.dataset as ds +import pytest + +from deltalake import convert_to_deltalake +from deltalake.exceptions import DeltaError +from deltalake.table import DeltaTable + + +def test_local_convert_to_delta(tmp_path: pathlib.Path, sample_data: pa.Table): + ds.write_dataset( + sample_data, + tmp_path, + format="parquet", + existing_data_behavior="overwrite_or_ignore", + ) + + name = "converted_table" + description = "parquet table converted to delta table with delta-rs" + convert_to_deltalake( + tmp_path, + name=name, + description=description, + configuration={"delta.AppendOnly": "True"}, + ) + + dt = DeltaTable(tmp_path) + + assert dt.version() == 0 + assert dt.files() == ["part-0.parquet"] + assert dt.metadata().name == name + assert dt.metadata().description == description + assert dt.metadata().configuration == {"delta.AppendOnly": "True"} + + +def test_convert_delta_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): + ds.write_dataset( + sample_data, + tmp_path, + format="parquet", + existing_data_behavior="overwrite_or_ignore", + ) + + convert_to_deltalake( + tmp_path, + ) + + with pytest.raises(DeltaError): + convert_to_deltalake( + tmp_path, + ) + + convert_to_deltalake(tmp_path, mode="ignore") + + +def test_convert_delta_with_partitioning(tmp_path: pathlib.Path, sample_data: pa.Table): + ds.write_dataset( + sample_data, + tmp_path, + format="parquet", + existing_data_behavior="overwrite_or_ignore", + partitioning=["utf8"], + partitioning_flavor="hive", + ) + + with pytest.raises( + DeltaError, + match="Generic error: The schema of partition columns must be provided to convert a Parquet table to a Delta table", + ): + convert_to_deltalake( + tmp_path, + ) + with pytest.raises( + ValueError, match="Partition strategy has to be provided with partition_by" + ): + convert_to_deltalake( + tmp_path, + partition_by=pa.schema([pa.field("utf8", pa.string())]), + ) + + with pytest.raises( + ValueError, + match="Currently only `hive` partition strategy is supported to be converted.", + ): + convert_to_deltalake( + tmp_path, + partition_by=pa.schema([pa.field("utf8", pa.string())]), + partition_strategy="directory", + ) + + convert_to_deltalake( + tmp_path, + partition_by=pa.schema([pa.field("utf8", pa.string())]), + partition_strategy="hive", + ) From 635313f759b9cb2149d264cc3941a174d6bde944 Mon Sep 17 00:00:00 2001 From: David Blajda Date: Sun, 19 Nov 2023 18:52:55 -0500 Subject: [PATCH 16/35] refactor: merge to use logical plans (#1720) # Description This refactors the merge operation to use DataFusion's DataFrame and LogicalPlan APIs The NLJ is eliminated and the query planner can pick the optimal join operator. This also enables the operation to use multiple threads and should result in significant speed up. Merge is still limited to using a single thread in some area. When collecting benchmarks, I encountered multiple OoM issues with Datafusion's hash join implementation. There are multiple tickets upstream open regarding this. For now, I've limited the number of partitions to just 1 to prevent this. Predicates passed as SQL are also easier to use now. Manual casting was required to ensure data types were aligned. Now the logical plan will perform type coercion when optimizing the plan. # Related Issues - enhances #850 - closes #1790 - closes #1753 --- .../src/delta_datafusion/logical.rs | 48 ++ .../src/delta_datafusion/mod.rs | 9 +- .../src/delta_datafusion/physical.rs | 180 +++++ crates/deltalake-core/src/operations/merge.rs | 631 +++++++++--------- crates/deltalake-core/src/operations/mod.rs | 136 +--- .../deltalake-core/src/operations/update.rs | 5 +- 6 files changed, 536 insertions(+), 473 deletions(-) create mode 100644 crates/deltalake-core/src/delta_datafusion/logical.rs create mode 100644 crates/deltalake-core/src/delta_datafusion/physical.rs diff --git a/crates/deltalake-core/src/delta_datafusion/logical.rs b/crates/deltalake-core/src/delta_datafusion/logical.rs new file mode 100644 index 0000000000..7b05dd57d9 --- /dev/null +++ b/crates/deltalake-core/src/delta_datafusion/logical.rs @@ -0,0 +1,48 @@ +//! Logical Operations for DataFusion + +use datafusion_expr::{LogicalPlan, UserDefinedLogicalNodeCore}; + +// Metric Observer is used to update DataFusion metrics from a record batch. +// See MetricObserverExec for the physical implementation + +#[derive(Debug, Hash, Eq, PartialEq)] +pub(crate) struct MetricObserver { + // id is preserved during conversion to physical node + pub id: String, + pub input: LogicalPlan, +} + +impl UserDefinedLogicalNodeCore for MetricObserver { + // Predicate push down is not supported for this node. Try to limit usage + // near the end of plan. + fn name(&self) -> &str { + "MetricObserver" + } + + fn inputs(&self) -> Vec<&datafusion_expr::LogicalPlan> { + vec![&self.input] + } + + fn schema(&self) -> &datafusion_common::DFSchemaRef { + self.input.schema() + } + + fn expressions(&self) -> Vec { + vec![] + } + + fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "MetricObserver id={}", &self.id) + } + + fn from_template( + &self, + _exprs: &[datafusion_expr::Expr], + inputs: &[datafusion_expr::LogicalPlan], + ) -> Self { + MetricObserver { + id: self.id.clone(), + input: inputs[0].clone(), + } + } +} diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 1410efbfbc..8dea811383 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -81,6 +81,8 @@ use crate::{open_table, open_table_with_storage_options, DeltaTable}; const PATH_COLUMN: &str = "__delta_rs_path"; pub mod expr; +pub mod logical; +pub mod physical; impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { @@ -351,7 +353,7 @@ pub(crate) fn logical_schema( snapshot: &DeltaTableState, scan_config: &DeltaScanConfig, ) -> DeltaResult { - let input_schema = snapshot.input_schema()?; + let input_schema = snapshot.arrow_schema()?; let mut fields = Vec::new(); for field in input_schema.fields.iter() { fields.push(field.to_owned()); @@ -505,11 +507,6 @@ impl<'a> DeltaScanBuilder<'a> { self } - pub fn with_schema(mut self, schema: SchemaRef) -> Self { - self.schema = Some(schema); - self - } - pub async fn build(self) -> DeltaResult { let config = self.config; let schema = match self.schema { diff --git a/crates/deltalake-core/src/delta_datafusion/physical.rs b/crates/deltalake-core/src/delta_datafusion/physical.rs new file mode 100644 index 0000000000..954df0b046 --- /dev/null +++ b/crates/deltalake-core/src/delta_datafusion/physical.rs @@ -0,0 +1,180 @@ +//! Physical Operations for DataFusion +use std::sync::Arc; + +use arrow_schema::SchemaRef; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::error::Result as DataFusionResult; +use datafusion::physical_plan::DisplayAs; +use datafusion::physical_plan::{ + metrics::{ExecutionPlanMetricsSet, MetricsSet}, + ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, +}; +use futures::{Stream, StreamExt}; + +use crate::DeltaTableError; + +// Metric Observer is used to update DataFusion metrics from a record batch. +// Typically the null count for a particular column is pulled after performing a +// projection since this count is easy to obtain + +pub(crate) type MetricObserverFunction = fn(&RecordBatch, &ExecutionPlanMetricsSet) -> (); + +pub(crate) struct MetricObserverExec { + parent: Arc, + id: String, + metrics: ExecutionPlanMetricsSet, + update: MetricObserverFunction, +} + +impl MetricObserverExec { + pub fn new(id: String, parent: Arc, f: MetricObserverFunction) -> Self { + MetricObserverExec { + parent, + id, + metrics: ExecutionPlanMetricsSet::new(), + update: f, + } + } + + pub fn try_new( + id: String, + inputs: &[Arc], + f: MetricObserverFunction, + ) -> DataFusionResult> { + match inputs { + [input] => Ok(Arc::new(MetricObserverExec::new(id, input.clone(), f))), + _ => Err(datafusion_common::DataFusionError::External(Box::new( + DeltaTableError::Generic("MetricObserverExec expects only one child".into()), + ))), + } + } + + pub fn id(&self) -> &str { + &self.id + } +} + +impl std::fmt::Debug for MetricObserverExec { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("MetricObserverExec") + .field("id", &self.id) + .field("metrics", &self.metrics) + .finish() + } +} + +impl DisplayAs for MetricObserverExec { + fn fmt_as( + &self, + _: datafusion::physical_plan::DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + write!(f, "MetricObserverExec id={}", self.id) + } +} + +impl ExecutionPlan for MetricObserverExec { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> arrow_schema::SchemaRef { + self.parent.schema() + } + + fn output_partitioning(&self) -> datafusion::physical_plan::Partitioning { + self.parent.output_partitioning() + } + + fn output_ordering(&self) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { + self.parent.output_ordering() + } + + fn children(&self) -> Vec> { + vec![self.parent.clone()] + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> datafusion_common::Result { + let res = self.parent.execute(partition, context)?; + Ok(Box::pin(MetricObserverStream { + schema: self.schema(), + input: res, + metrics: self.metrics.clone(), + update: self.update, + })) + } + + fn statistics(&self) -> DataFusionResult { + self.parent.statistics() + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + MetricObserverExec::try_new(self.id.clone(), &children, self.update) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } +} + +struct MetricObserverStream { + schema: SchemaRef, + input: SendableRecordBatchStream, + metrics: ExecutionPlanMetricsSet, + update: MetricObserverFunction, +} + +impl Stream for MetricObserverStream { + type Item = DataFusionResult; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + self.input.poll_next_unpin(cx).map(|x| match x { + Some(Ok(batch)) => { + (self.update)(&batch, &self.metrics); + Some(Ok(batch)) + } + other => other, + }) + } + + fn size_hint(&self) -> (usize, Option) { + self.input.size_hint() + } +} + +impl RecordBatchStream for MetricObserverStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +pub(crate) fn find_metric_node( + id: &str, + parent: &Arc, +) -> Option> { + //! Used to locate the physical MetricCountExec Node after the planner converts the logical node + if let Some(metric) = parent.as_any().downcast_ref::() { + if metric.id().eq(id) { + return Some(parent.to_owned()); + } + } + + for child in &parent.children() { + let res = find_metric_node(id, child); + if res.is_some() { + return res; + } + } + + None +} diff --git a/crates/deltalake-core/src/operations/merge.rs b/crates/deltalake-core/src/operations/merge.rs index a9ad6a8655..8b0dd56708 100644 --- a/crates/deltalake-core/src/operations/merge.rs +++ b/crates/deltalake-core/src/operations/merge.rs @@ -8,8 +8,7 @@ //! specified matter. See [`MergeBuilder`] for more information //! //! *WARNING* The current implementation rewrites the entire delta table so only -//! use on small to medium sized tables. The solution also cannot take advantage -//! of multiple threads and is limited to a single single thread. +//! use on small to medium sized tables. //! Enhancements tracked at #850 //! //! # Example @@ -37,27 +36,25 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::{Instant, SystemTime, UNIX_EPOCH}; -use arrow_schema::SchemaRef; +use async_trait::async_trait; +use datafusion::datasource::provider_as_source; use datafusion::error::Result as DataFusionResult; +use datafusion::execution::context::{QueryPlanner, SessionConfig}; use datafusion::logical_expr::build_join_schema; -use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner}; use datafusion::{ execution::context::SessionState, physical_plan::{ - filter::FilterExec, - joins::{ - utils::{build_join_schema as physical_build_join_schema, JoinFilter}, - NestedLoopJoinExec, - }, metrics::{MetricBuilder, MetricsSet}, - projection::ProjectionExec, ExecutionPlan, }, prelude::{DataFrame, SessionContext}, }; -use datafusion_common::{Column, DFField, DFSchema, ScalarValue, TableReference}; +use datafusion_common::{Column, DFSchema, ScalarValue, TableReference}; use datafusion_expr::{col, conditional_expressions::CaseBuilder, lit, when, Expr, JoinType}; -use datafusion_physical_expr::{create_physical_expr, expressions, PhysicalExpr}; +use datafusion_expr::{ + Extension, LogicalPlan, LogicalPlanBuilder, UserDefinedLogicalNode, UNNAMED_TABLE, +}; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; use serde::Serialize; @@ -66,15 +63,19 @@ use serde_json::Value; use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; use super::transaction::{commit, PROTOCOL}; use crate::delta_datafusion::expr::{fmt_expr_to_sql, parse_predicate_expression}; -use crate::delta_datafusion::{register_store, DeltaScanBuilder}; +use crate::delta_datafusion::logical::MetricObserver; +use crate::delta_datafusion::physical::{find_metric_node, MetricObserverExec}; +use crate::delta_datafusion::{register_store, DeltaScanConfig, DeltaTableProvider}; use crate::kernel::{Action, Remove}; use crate::logstore::LogStoreRef; -use crate::operations::datafusion_utils::MetricObserverExec; use crate::operations::write::write_execution_plan; use crate::protocol::{DeltaOperation, MergePredicate}; use crate::table::state::DeltaTableState; use crate::{DeltaResult, DeltaTable, DeltaTableError}; +const SOURCE_COLUMN: &str = "__delta_rs_source"; +const TARGET_COLUMN: &str = "__delta_rs_target"; + const OPERATION_COLUMN: &str = "__delta_rs_operation"; const DELETE_COLUMN: &str = "__delta_rs_delete"; const TARGET_INSERT_COLUMN: &str = "__delta_rs_target_insert"; @@ -83,11 +84,16 @@ const TARGET_DELETE_COLUMN: &str = "__delta_rs_target_delete"; const TARGET_COPY_COLUMN: &str = "__delta_rs_target_copy"; const SOURCE_COUNT_METRIC: &str = "num_source_rows"; +const TARGET_COUNT_METRIC: &str = "num_target_rows"; const TARGET_COPY_METRIC: &str = "num_copied_rows"; const TARGET_INSERTED_METRIC: &str = "num_target_inserted_rows"; const TARGET_UPDATED_METRIC: &str = "num_target_updated_rows"; const TARGET_DELETED_METRIC: &str = "num_target_deleted_rows"; +const SOURCE_COUNT_ID: &str = "merge_source_count"; +const TARGET_COUNT_ID: &str = "merge_target_count"; +const OUTPUT_COUNT_ID: &str = "merge_output_count"; + /// Merge records into a Delta Table. pub struct MergeBuilder { /// The join predicate @@ -557,6 +563,89 @@ pub struct MergeMetrics { pub rewrite_time_ms: u64, } +struct MergeMetricExtensionPlanner {} + +#[async_trait] +impl ExtensionPlanner for MergeMetricExtensionPlanner { + async fn plan_extension( + &self, + _planner: &dyn PhysicalPlanner, + node: &dyn UserDefinedLogicalNode, + _logical_inputs: &[&LogicalPlan], + physical_inputs: &[Arc], + _session_state: &SessionState, + ) -> DataFusionResult>> { + if let Some(metric_observer) = node.as_any().downcast_ref::() { + if metric_observer.id.eq(SOURCE_COUNT_ID) { + return Ok(Some(MetricObserverExec::try_new( + SOURCE_COUNT_ID.into(), + physical_inputs, + |batch, metrics| { + MetricBuilder::new(metrics) + .global_counter(SOURCE_COUNT_METRIC) + .add(batch.num_rows()); + }, + )?)); + } + + if metric_observer.id.eq(TARGET_COUNT_ID) { + return Ok(Some(MetricObserverExec::try_new( + TARGET_COUNT_ID.into(), + physical_inputs, + |batch, metrics| { + MetricBuilder::new(metrics) + .global_counter(TARGET_COUNT_METRIC) + .add(batch.num_rows()); + }, + )?)); + } + + if metric_observer.id.eq(OUTPUT_COUNT_ID) { + return Ok(Some(MetricObserverExec::try_new( + OUTPUT_COUNT_ID.into(), + physical_inputs, + |batch, metrics| { + MetricBuilder::new(metrics) + .global_counter(TARGET_INSERTED_METRIC) + .add( + batch + .column_by_name(TARGET_INSERT_COLUMN) + .unwrap() + .null_count(), + ); + MetricBuilder::new(metrics) + .global_counter(TARGET_UPDATED_METRIC) + .add( + batch + .column_by_name(TARGET_UPDATE_COLUMN) + .unwrap() + .null_count(), + ); + MetricBuilder::new(metrics) + .global_counter(TARGET_DELETED_METRIC) + .add( + batch + .column_by_name(TARGET_DELETE_COLUMN) + .unwrap() + .null_count(), + ); + MetricBuilder::new(metrics) + .global_counter(TARGET_COPY_METRIC) + .add( + batch + .column_by_name(TARGET_COPY_COLUMN) + .unwrap() + .null_count(), + ); + }, + )?)); + } + } + + Ok(None) + } +} + #[allow(clippy::too_many_arguments)] async fn execute( predicate: Expression, @@ -589,83 +678,61 @@ async fn execute( // If the user specified any not_source_match operations then those // predicates also need to be considered when pruning - let target = Arc::new( - DeltaScanBuilder::new(snapshot, log_store.clone(), &state) - .with_schema(snapshot.input_schema()?) - .build() - .await?, - ); - - let source = source.create_physical_plan().await?; - - let source_count = Arc::new(MetricObserverExec::new(source, |batch, metrics| { - MetricBuilder::new(metrics) - .global_counter(SOURCE_COUNT_METRIC) - .add(batch.num_rows()); - })); - - let mut expressions: Vec<(Arc, String)> = Vec::new(); - let source_schema = source_count.schema(); - - for (i, field) in source_schema.fields().into_iter().enumerate() { - expressions.push(( - Arc::new(expressions::Column::new(field.name(), i)), - field.name().clone(), - )); - } - expressions.push(( - Arc::new(expressions::Literal::new(true.into())), - "__delta_rs_source".to_owned(), - )); - let source = Arc::new(ProjectionExec::try_new(expressions, source_count.clone())?); - - let mut expressions: Vec<(Arc, String)> = Vec::new(); - let target_schema = target.schema(); - for (i, field) in target_schema.fields().into_iter().enumerate() { - expressions.push(( - Arc::new(expressions::Column::new(field.name(), i)), - field.name().to_owned(), - )); - } - expressions.push(( - Arc::new(expressions::Literal::new(true.into())), - "__delta_rs_target".to_owned(), - )); - let target = Arc::new(ProjectionExec::try_new(expressions, target.clone())?); - - // TODO: Currently a NestedLoopJoin is used but we should target to support SortMergeJoin - // This would require rewriting the join predicate to only contain equality between left and right columns and pushing some filters down - // Ideally it would be nice if the optimizer / planner can pick the best join so maybe explore rewriting the entire operation using logical plans. - - // NLJ requires both sides to have one partition for outer joins - let target = Arc::new(CoalescePartitionsExec::new(target)); - let source = Arc::new(CoalescePartitionsExec::new(source)); - - let source_schema = match &source_alias { - Some(alias) => { - DFSchema::try_from_qualified_schema(TableReference::bare(alias), &source.schema())? - } - None => DFSchema::try_from(source.schema().as_ref().to_owned())?, + let source_name = match &source_alias { + Some(alias) => TableReference::bare(alias.to_string()), + None => TableReference::bare(UNNAMED_TABLE), }; - let target_schema = match &target_alias { - Some(alias) => { - DFSchema::try_from_qualified_schema(TableReference::bare(alias), &target.schema())? - } - None => DFSchema::try_from(target.schema().as_ref().to_owned())?, + let target_name = match &target_alias { + Some(alias) => TableReference::bare(alias.to_string()), + None => TableReference::bare(UNNAMED_TABLE), }; - let join_schema_df = build_join_schema(&source_schema, &target_schema, &JoinType::Full)?; + // This is only done to provide the source columns with a correct table reference. Just renaming the columns does not work + let source = + LogicalPlanBuilder::scan(source_name, provider_as_source(source.into_view()), None)? + .build()?; + + let source = LogicalPlan::Extension(Extension { + node: Arc::new(MetricObserver { + id: SOURCE_COUNT_ID.into(), + input: source, + }), + }); + + let source = DataFrame::new(state.clone(), source); + let source = source.with_column(SOURCE_COLUMN, lit(true))?; + + let target_provider = Arc::new(DeltaTableProvider::try_new( + snapshot.clone(), + log_store.clone(), + DeltaScanConfig::default(), + )?); + let target_provider = provider_as_source(target_provider); + + let target = LogicalPlanBuilder::scan(target_name, target_provider, None)?.build()?; - let join_schema = - physical_build_join_schema(&source.schema(), &target.schema(), &JoinType::Full); - let (join_schema, join_order) = (join_schema.0, join_schema.1); + // TODO: This is here to prevent predicate pushdowns. In the future we can replace this node to allow pushdowns depending on which operations are being used. + let target = LogicalPlan::Extension(Extension { + node: Arc::new(MetricObserver { + id: TARGET_COUNT_ID.into(), + input: target, + }), + }); + let target = DataFrame::new(state.clone(), target); + let target = target.with_column(TARGET_COLUMN, lit(true))?; + let source_schema = source.schema(); + let target_schema = target.schema(); + let join_schema_df = build_join_schema(source_schema, target_schema, &JoinType::Full)?; let predicate = match predicate { Expression::DataFusion(expr) => expr, Expression::String(s) => parse_predicate_expression(&join_schema_df, s, &state)?, }; + let join = source.join(target, JoinType::Full, &[], &[], Some(predicate.clone()))?; + let join_schema_df = join.schema().to_owned(); + let match_operations: Vec = match_operations .into_iter() .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) @@ -681,40 +748,15 @@ async fn execute( .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) .collect::, DeltaTableError>>()?; - let predicate_expr = create_physical_expr( - &predicate, - &join_schema_df, - &join_schema, - state.execution_props(), - )?; - - let join_filter = JoinFilter::new(predicate_expr, join_order, join_schema); - let join: Arc = Arc::new(NestedLoopJoinExec::try_new( - source.clone(), - target.clone(), - Some(join_filter), - &datafusion_expr::JoinType::Full, - )?); - - // Project to include __delta_rs_operation which indicates which particular operation to perform on the column. - let mut expressions: Vec<(Arc, String)> = Vec::new(); - let schema = join.schema(); - for (i, field) in schema.fields().into_iter().enumerate() { - expressions.push(( - Arc::new(expressions::Column::new(field.name(), i)), - field.name().to_owned(), - )); - } - - let matched = col("__delta_rs_source") + let matched = col(SOURCE_COLUMN) .is_true() - .and(col("__delta_rs_target").is_true()); - let not_matched_target = col("__delta_rs_source") + .and(col(TARGET_COLUMN).is_true()); + let not_matched_target = col(SOURCE_COLUMN) .is_true() - .and(col("__delta_rs_target").is_null()); - let not_matched_source = col("__delta_rs_source") + .and(col(TARGET_COLUMN).is_null()); + let not_matched_source = col(SOURCE_COLUMN) .is_null() - .and(col("__delta_rs_target")) + .and(col(TARGET_COLUMN)) .is_true(); // Plus 3 for the default operations for each match category @@ -811,35 +853,10 @@ async fn execute( let case = CaseBuilder::new(None, when_expr, then_expr, None).end()?; - let case = create_physical_expr( - &case, - &join_schema_df, - &join.schema(), - state.execution_props(), - )?; - expressions.push((case, OPERATION_COLUMN.to_owned())); - let projection = Arc::new(ProjectionExec::try_new(expressions, join.clone())?); - - let mut f = join_schema_df.fields().to_owned(); - f.push(DFField::new_unqualified( - OPERATION_COLUMN, - arrow_schema::DataType::Int64, - false, - )); - let project_schema_df = DFSchema::new_with_metadata(f, HashMap::new())?; - - // Project again and include the original table schema plus a column to mark if row needs to be filtered before write - let mut expressions: Vec<(Arc, String)> = Vec::new(); - let schema = projection.schema(); - for (i, field) in schema.fields().into_iter().enumerate() { - expressions.push(( - Arc::new(expressions::Column::new(field.name(), i)), - field.name().to_owned(), - )); - } + let projection = join.with_column(OPERATION_COLUMN, case)?; - let mut projection_map = HashMap::new(); - let mut f = project_schema_df.fields().clone(); + let mut new_columns = projection; + let mut write_projection = Vec::new(); for delta_field in snapshot.schema().unwrap().fields() { let mut when_expr = Vec::with_capacity(operations_size); @@ -853,7 +870,6 @@ async fn execute( }; let name = delta_field.name(); let column = Column::new(qualifier.clone(), name); - let field = project_schema_df.field_with_name(qualifier.as_ref(), name)?; for (idx, (operations, _)) in ops.iter().enumerate() { let op = operations @@ -873,22 +889,9 @@ async fn execute( ) .end()?; - let case = create_physical_expr( - &case, - &project_schema_df, - &projection.schema(), - state.execution_props(), - )?; - - projection_map.insert(delta_field.name(), expressions.len()); let name = "__delta_rs_c_".to_owned() + delta_field.name(); - - f.push(DFField::new_unqualified( - &name, - field.data_type().clone(), - true, - )); - expressions.push((case, name)); + write_projection.push(col(name.clone()).alias(delta_field.name())); + new_columns = new_columns.with_column(&name, case)?; } let mut insert_when = Vec::with_capacity(ops.len()); @@ -954,168 +957,47 @@ async fn execute( ); } - fn build_case( - when: Vec, - then: Vec, - schema: SchemaRef, - input_dfschema: &DFSchema, - state: &SessionState, - ) -> DataFusionResult> { - let case = CaseBuilder::new( + fn build_case(when: Vec, then: Vec) -> DataFusionResult { + CaseBuilder::new( Some(Box::new(col(OPERATION_COLUMN))), when, then, Some(Box::new(lit(false))), ) - .end()?; - - create_physical_expr(&case, input_dfschema, &schema, state.execution_props()) + .end() } - let schema = projection.schema(); - let input_dfschema = project_schema_df; - expressions.push(( - build_case( - delete_when, - delete_then, - schema.clone(), - &input_dfschema, - &state, - )?, - DELETE_COLUMN.to_owned(), - )); - f.push(DFField::new_unqualified( - DELETE_COLUMN, - arrow_schema::DataType::Boolean, - true, - )); - - expressions.push(( - build_case( - insert_when, - insert_then, - schema.clone(), - &input_dfschema, - &state, - )?, - TARGET_INSERT_COLUMN.to_owned(), - )); - f.push(DFField::new_unqualified( - TARGET_INSERT_COLUMN, - arrow_schema::DataType::Boolean, - true, - )); - - expressions.push(( - build_case( - update_when, - update_then, - schema.clone(), - &input_dfschema, - &state, - )?, - TARGET_UPDATE_COLUMN.to_owned(), - )); - f.push(DFField::new_unqualified( - TARGET_UPDATE_COLUMN, - arrow_schema::DataType::Boolean, - true, - )); - - expressions.push(( - build_case( - target_delete_when, - target_delete_then, - schema.clone(), - &input_dfschema, - &state, - )?, - TARGET_DELETE_COLUMN.to_owned(), - )); - f.push(DFField::new_unqualified( + new_columns = new_columns.with_column(DELETE_COLUMN, build_case(delete_when, delete_then)?)?; + new_columns = + new_columns.with_column(TARGET_INSERT_COLUMN, build_case(insert_when, insert_then)?)?; + new_columns = + new_columns.with_column(TARGET_UPDATE_COLUMN, build_case(update_when, update_then)?)?; + new_columns = new_columns.with_column( TARGET_DELETE_COLUMN, - arrow_schema::DataType::Boolean, - true, - )); - - expressions.push(( - build_case( - copy_when, - copy_then, - schema.clone(), - &input_dfschema, - &state, - )?, - TARGET_COPY_COLUMN.to_owned(), - )); - f.push(DFField::new_unqualified( - TARGET_COPY_COLUMN, - arrow_schema::DataType::Boolean, - true, - )); - - let projection = Arc::new(ProjectionExec::try_new(expressions, projection.clone())?); - - let target_count_plan = Arc::new(MetricObserverExec::new(projection, |batch, metrics| { - MetricBuilder::new(metrics) - .global_counter(TARGET_INSERTED_METRIC) - .add( - batch - .column_by_name(TARGET_INSERT_COLUMN) - .unwrap() - .null_count(), - ); - MetricBuilder::new(metrics) - .global_counter(TARGET_UPDATED_METRIC) - .add( - batch - .column_by_name(TARGET_UPDATE_COLUMN) - .unwrap() - .null_count(), - ); - MetricBuilder::new(metrics) - .global_counter(TARGET_DELETED_METRIC) - .add( - batch - .column_by_name(TARGET_DELETE_COLUMN) - .unwrap() - .null_count(), - ); - MetricBuilder::new(metrics) - .global_counter(TARGET_COPY_METRIC) - .add( - batch - .column_by_name(TARGET_COPY_COLUMN) - .unwrap() - .null_count(), - ); - })); - - let write_schema_df = DFSchema::new_with_metadata(f, HashMap::new())?; - - let write_predicate = create_physical_expr( - &(col(DELETE_COLUMN).is_false()), - &write_schema_df, - &target_count_plan.schema(), - state.execution_props(), + build_case(target_delete_when, target_delete_then)?, )?; - let filter: Arc = Arc::new(FilterExec::try_new( - write_predicate, - target_count_plan.clone(), - )?); + new_columns = new_columns.with_column(TARGET_COPY_COLUMN, build_case(copy_when, copy_then)?)?; - let mut expressions: Vec<(Arc, String)> = Vec::new(); - for (key, value) in projection_map { - expressions.push(( - Arc::new(expressions::Column::new( - &("__delta_rs_c_".to_owned() + key), - value, - )), - key.to_owned(), - )); - } - // project filtered records to delta schema - let projection = Arc::new(ProjectionExec::try_new(expressions, filter.clone())?); + let new_columns = new_columns.into_optimized_plan()?; + let operation_count = LogicalPlan::Extension(Extension { + node: Arc::new(MetricObserver { + id: OUTPUT_COUNT_ID.into(), + input: new_columns, + }), + }); + + let operation_count = DataFrame::new(state.clone(), operation_count); + let filtered = operation_count.filter(col(DELETE_COLUMN).is_false())?; + + let project = filtered.select(write_projection)?; + let optimized = &project.into_optimized_plan()?; + + let state = state.with_query_planner(Arc::new(MergePlanner {})); + let write = state.create_physical_plan(optimized).await?; + + let err = || DeltaTableError::Generic("Unable to locate expected metric node".into()); + let source_count = find_metric_node(SOURCE_COUNT_ID, &write).ok_or_else(err)?; + let op_count = find_metric_node(OUTPUT_COUNT_ID, &write).ok_or_else(err)?; // write projected records let table_partition_cols = current_metadata.partition_columns.clone(); @@ -1124,9 +1006,9 @@ async fn execute( let add_actions = write_execution_plan( snapshot, state.clone(), - projection.clone(), + write, table_partition_cols.clone(), - log_store.object_store().clone(), + log_store.object_store(), Some(snapshot.table_config().target_file_size() as usize), None, writer_properties, @@ -1163,7 +1045,7 @@ async fn execute( let mut version = snapshot.version(); let source_count_metrics = source_count.metrics().unwrap(); - let target_count_metrics = target_count_plan.metrics().unwrap(); + let target_count_metrics = op_count.metrics().unwrap(); fn get_metric(metrics: &MetricsSet, name: &str) -> usize { metrics.sum_by_name(name).map(|m| m.as_usize()).unwrap_or(0) } @@ -1200,6 +1082,25 @@ async fn execute( Ok(((actions, version), metrics)) } +// TODO: Abstract MergePlanner into DeltaPlanner to support other delta operations in the future. +struct MergePlanner {} + +#[async_trait] +impl QueryPlanner for MergePlanner { + async fn create_physical_plan( + &self, + logical_plan: &LogicalPlan, + session_state: &SessionState, + ) -> DataFusionResult> { + let planner = Arc::new(Box::new(DefaultPhysicalPlanner::with_extension_planners( + vec![Arc::new(MergeMetricExtensionPlanner {})], + ))); + planner + .create_physical_plan(logical_plan, session_state) + .await + } +} + impl std::future::IntoFuture for MergeBuilder { type Output = DeltaResult<(DeltaTable, MergeMetrics)>; type IntoFuture = BoxFuture<'static, Self::Output>; @@ -1211,7 +1112,9 @@ impl std::future::IntoFuture for MergeBuilder { PROTOCOL.can_write_to(&this.snapshot)?; let state = this.state.unwrap_or_else(|| { - let session = SessionContext::new(); + //TODO: Datafusion's Hashjoin has some memory issues. Running with all cores results in a OoM. Can be removed when upstream improvemetns are made. + let config = SessionConfig::new().with_target_partitions(1); + let session = SessionContext::new_with_config(config); // If a user provides their own their DF state then they must register the store themselves register_store(this.log_store.clone(), session.runtime_env()); @@ -1349,8 +1252,8 @@ mod tests { async fn assert_merge(table: DeltaTable, metrics: MergeMetrics) { assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 1); - assert_eq!(metrics.num_target_files_added, 1); + assert!(table.get_file_uris().count() >= 1); + assert!(metrics.num_target_files_added >= 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); assert_eq!(metrics.num_target_rows_updated, 3); @@ -1442,7 +1345,7 @@ mod tests { .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate("target.value = arrow_cast(1, 'Int32')") + .predicate("target.value = 1") .update("value", "target.value + cast(1 as int)") }) .unwrap() @@ -1470,9 +1373,7 @@ mod tests { ); assert_eq!( parameters["notMatchedBySourcePredicates"], - json!( - r#"[{"actionType":"update","predicate":"target.value = arrow_cast(1, 'Int32')"}]"# - ) + json!(r#"[{"actionType":"update","predicate":"target.value = 1"}]"#) ); assert_merge(table, metrics).await; @@ -1500,9 +1401,7 @@ mod tests { }) .unwrap() .when_not_matched_by_source_update(|update| { - update - .predicate("value = arrow_cast(1, 'Int32')") - .update("value", "value + cast(1 as int)") + update.predicate("value = 1").update("value", "value + 1") }) .unwrap() .when_not_matched_insert(|insert| { @@ -1543,8 +1442,8 @@ mod tests { .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate("value = arrow_cast(1, 'Int32')") - .update("value", "target.value + cast(1 as int)") + .predicate("value = 1") + .update("value", "target.value + 1") }) .unwrap() .when_not_matched_insert(|insert| { @@ -1657,8 +1556,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 3); - assert_eq!(metrics.num_target_files_added, 3); + assert!(table.get_file_uris().count() >= 3); + assert!(metrics.num_target_files_added >= 3); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 1); assert_eq!(metrics.num_target_rows_updated, 3); @@ -1720,8 +1619,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); - assert_eq!(metrics.num_target_files_added, 2); + assert!(table.get_file_uris().count() >= 2); + assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 2); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1784,8 +1683,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); - assert_eq!(metrics.num_target_files_added, 2); + assert!(table.get_file_uris().count() >= 2); + assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 3); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1918,8 +1817,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert_eq!(table.get_file_uris().count(), 2); - assert_eq!(metrics.num_target_files_added, 2); + assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 3); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1949,4 +1847,77 @@ mod tests { let actual = get_data(&table).await; assert_batches_sorted_eq!(&expected, &actual); } + + #[tokio::test] + async fn test_merge_empty_table() { + let schema = get_arrow_schema(&None); + let table = setup_table(Some(vec!["modified"])).await; + + assert_eq!(table.version(), 0); + assert_eq!(table.get_file_uris().count(), 0); + + let ctx = SessionContext::new(); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["B", "C", "X"])), + Arc::new(arrow::array::Int32Array::from(vec![10, 20, 30])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2023-07-04", + "2023-07-04", + ])), + ], + ) + .unwrap(); + let source = ctx.read_batch(batch).unwrap(); + + let (table, metrics) = DeltaOps(table) + .merge( + source, + col("target.id") + .eq(col("source.id")) + .and(col("target.modified").eq(lit("2021-02-02"))), + ) + .with_source_alias("source") + .with_target_alias("target") + .when_matched_update(|update| { + update + .update("value", col("source.value")) + .update("modified", col("source.modified")) + }) + .unwrap() + .when_not_matched_insert(|insert| { + insert + .set("id", col("source.id")) + .set("value", col("source.value")) + .set("modified", col("source.modified")) + }) + .unwrap() + .await + .unwrap(); + + assert_eq!(table.version(), 1); + assert!(table.get_file_uris().count() >= 2); + assert!(metrics.num_target_files_added >= 2); + assert_eq!(metrics.num_target_files_removed, 0); + assert_eq!(metrics.num_target_rows_copied, 0); + assert_eq!(metrics.num_target_rows_updated, 0); + assert_eq!(metrics.num_target_rows_inserted, 3); + assert_eq!(metrics.num_target_rows_deleted, 0); + assert_eq!(metrics.num_output_rows, 3); + assert_eq!(metrics.num_source_rows, 3); + + let expected = vec![ + "+----+-------+------------+", + "| id | value | modified |", + "+----+-------+------------+", + "| B | 10 | 2021-02-02 |", + "| C | 20 | 2023-07-04 |", + "| X | 30 | 2023-07-04 |", + "+----+-------+------------+", + ]; + let actual = get_data(&table).await; + assert_batches_sorted_eq!(&expected, &actual); + } } diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index fb0f25d379..473ef1451b 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -209,20 +209,9 @@ impl AsRef for DeltaOps { #[cfg(feature = "datafusion")] mod datafusion_utils { - use std::sync::Arc; - - use arrow_schema::SchemaRef; - use datafusion::arrow::record_batch::RecordBatch; - use datafusion::error::Result as DataFusionResult; use datafusion::execution::context::SessionState; - use datafusion::physical_plan::DisplayAs; - use datafusion::physical_plan::{ - metrics::{ExecutionPlanMetricsSet, MetricsSet}, - ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, - }; - use datafusion_common::{DFSchema, Statistics}; + use datafusion_common::DFSchema; use datafusion_expr::Expr; - use futures::{Stream, StreamExt}; use crate::{delta_datafusion::expr::parse_predicate_expression, DeltaResult}; @@ -272,127 +261,4 @@ mod datafusion_utils { None => None, }) } - - pub(crate) type MetricObserverFunction = fn(&RecordBatch, &ExecutionPlanMetricsSet) -> (); - - pub(crate) struct MetricObserverExec { - parent: Arc, - metrics: ExecutionPlanMetricsSet, - update: MetricObserverFunction, - } - - impl MetricObserverExec { - pub fn new(parent: Arc, f: MetricObserverFunction) -> Self { - MetricObserverExec { - parent, - metrics: ExecutionPlanMetricsSet::new(), - update: f, - } - } - } - - impl std::fmt::Debug for MetricObserverExec { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("MergeStatsExec") - .field("parent", &self.parent) - .field("metrics", &self.metrics) - .finish() - } - } - - impl DisplayAs for MetricObserverExec { - fn fmt_as( - &self, - _: datafusion::physical_plan::DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "MetricObserverExec") - } - } - - impl ExecutionPlan for MetricObserverExec { - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn schema(&self) -> arrow_schema::SchemaRef { - self.parent.schema() - } - - fn output_partitioning(&self) -> datafusion::physical_plan::Partitioning { - self.parent.output_partitioning() - } - - fn output_ordering(&self) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { - self.parent.output_ordering() - } - - fn children(&self) -> Vec> { - vec![self.parent.clone()] - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> datafusion_common::Result - { - let res = self.parent.execute(partition, context)?; - Ok(Box::pin(MetricObserverStream { - schema: self.schema(), - input: res, - metrics: self.metrics.clone(), - update: self.update, - })) - } - - fn statistics(&self) -> DataFusionResult { - self.parent.statistics() - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> datafusion_common::Result> { - ExecutionPlan::with_new_children(self.parent.clone(), children) - } - - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } - } - - struct MetricObserverStream { - schema: SchemaRef, - input: SendableRecordBatchStream, - metrics: ExecutionPlanMetricsSet, - update: MetricObserverFunction, - } - - impl Stream for MetricObserverStream { - type Item = DataFusionResult; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - self.input.poll_next_unpin(cx).map(|x| match x { - Some(Ok(batch)) => { - (self.update)(&batch, &self.metrics); - Some(Ok(batch)) - } - other => other, - }) - } - - fn size_hint(&self) -> (usize, Option) { - self.input.size_hint() - } - } - - impl RecordBatchStream for MetricObserverStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - } } diff --git a/crates/deltalake-core/src/operations/update.rs b/crates/deltalake-core/src/operations/update.rs index 7583ed6b39..907dec5998 100644 --- a/crates/deltalake-core/src/operations/update.rs +++ b/crates/deltalake-core/src/operations/update.rs @@ -43,10 +43,10 @@ use parquet::file::properties::WriterProperties; use serde::Serialize; use serde_json::Value; -use super::datafusion_utils::{Expression, MetricObserverExec}; +use super::datafusion_utils::Expression; use super::transaction::{commit, PROTOCOL}; use super::write::write_execution_plan; -use crate::delta_datafusion::expr::fmt_expr_to_sql; +use crate::delta_datafusion::{expr::fmt_expr_to_sql, physical::MetricObserverExec}; use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::kernel::{Action, Remove}; use crate::logstore::LogStoreRef; @@ -275,6 +275,7 @@ async fn execute( Arc::new(ProjectionExec::try_new(expressions, scan)?); let count_plan = Arc::new(MetricObserverExec::new( + "update_count".into(), projection_predicate.clone(), |batch, metrics| { let array = batch.column_by_name("__delta_rs_update_predicate").unwrap(); From 633fd7f2966eef6d8a98dc0785d06c6f8ff8a1e0 Mon Sep 17 00:00:00 2001 From: David Blajda Date: Mon, 20 Nov 2023 02:40:14 -0500 Subject: [PATCH 17/35] feat: create benchmarks for merge (#1857) # Description Implements benchmarks that are similar to Spark's Delta benchmarks. Enable us to have a standard benchmark to measure improvements to merge and some pieces can be factored out to build a framework for bench marking delta workflows. --- crates/benchmarks/Cargo.toml | 46 ++ crates/benchmarks/README.md | 55 +++ crates/benchmarks/src/bin/merge.rs | 647 +++++++++++++++++++++++++++++ 3 files changed, 748 insertions(+) create mode 100644 crates/benchmarks/Cargo.toml create mode 100644 crates/benchmarks/README.md create mode 100644 crates/benchmarks/src/bin/merge.rs diff --git a/crates/benchmarks/Cargo.toml b/crates/benchmarks/Cargo.toml new file mode 100644 index 0000000000..76bcc8a312 --- /dev/null +++ b/crates/benchmarks/Cargo.toml @@ -0,0 +1,46 @@ +[package] +name = "delta-benchmarks" +version = "0.0.1" +authors = ["David Blajda "] +homepage = "https://github.com/delta-io/delta.rs" +license = "Apache-2.0" +keywords = ["deltalake", "delta", "datalake"] +description = "Delta-rs Benchmarks" +edition = "2021" + +[dependencies] +clap = { version = "4", features = [ "derive" ] } +chrono = { version = "0.4.31", default-features = false, features = ["clock"] } +tokio = { version = "1", features = ["fs", "macros", "rt", "io-util"] } +env_logger = "0" + +# arrow +arrow = { workspace = true } +arrow-array = { workspace = true } +arrow-buffer = { workspace = true } +arrow-cast = { workspace = true } +arrow-ord = { workspace = true } +arrow-row = { workspace = true } +arrow-schema = { workspace = true, features = ["serde"] } +arrow-select = { workspace = true } +parquet = { workspace = true, features = [ + "async", + "object_store", +] } + +# serde +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true } + +# datafusion +datafusion = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-common = { workspace = true } +datafusion-proto = { workspace = true } +datafusion-sql = { workspace = true } +datafusion-physical-expr = { workspace = true } + +[dependencies.deltalake-core] +path = "../deltalake-core" +version = "0" +features = ["datafusion"] diff --git a/crates/benchmarks/README.md b/crates/benchmarks/README.md new file mode 100644 index 0000000000..c5d6b0b920 --- /dev/null +++ b/crates/benchmarks/README.md @@ -0,0 +1,55 @@ +# Merge +The merge benchmarks are similar to the ones used by [Delta Spark](https://github.com/delta-io/delta/pull/1835). + + +## Dataset + +Databricks maintains a public S3 bucket of the TPC-DS dataset with various factor where requesters must pay to download this dataset. Below is an example of how to list the 1gb scale factor + +``` +aws s3api list-objects --bucket devrel-delta-datasets --request-payer requester --prefix tpcds-2.13/tpcds_sf1_parquet/web_returns/ +``` + +You can generate the TPC-DS dataset yourself by downloading and compiling [the generator](https://www.tpc.org/tpc_documents_current_versions/current_specifications5.asp) +You may need to update the CFLAGS to include `-fcommon` to compile on newer versions of GCC. + +## Commands +These commands can be executed from the root of the benchmark crate. Some commands depend on the existance of the TPC-DS Dataset existing. + +### Convert +Converts a TPC-DS web_returns csv into a Delta table +Assumes the dataset is pipe delimited and records do not have a trailing delimiter + +``` + cargo run --release --bin merge -- convert data/tpcds/web_returns.dat data/web_returns +``` + +### Standard +Execute the standard merge bench suite. +Results can be saved to a delta table for further analysis. +This table has the following schema: + +group_id: Used to group all tests that executed as a part of this call. Default value is the timestamp of execution +name: The benchmark name that was executed +sample: The iteration number for a given benchmark name +duration_ms: How long the benchmark took in ms +data: Free field to pack any additonal data + +``` + cargo run --release --bin merge -- standard data/web_returns 1 data/merge_results +``` + +### Compare +Compare the results of two different runs. +The a Delta table paths and the `group_id` of each run and obtain the speedup for each test case + +``` + cargo run --release --bin merge -- compare data/benchmarks/ 1698636172801 data/benchmarks/ 1699759539902 +``` + +### Show +Show all benchmarks results from a delta table + +``` + cargo run --release --bin merge -- show data/benchmark +``` diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs new file mode 100644 index 0000000000..5afa3e6f35 --- /dev/null +++ b/crates/benchmarks/src/bin/merge.rs @@ -0,0 +1,647 @@ +use std::{ + sync::Arc, + time::{SystemTime, UNIX_EPOCH}, +}; + +use arrow::datatypes::Schema as ArrowSchema; +use arrow_array::{RecordBatch, StringArray, UInt32Array}; +use chrono::Duration; +use clap::{command, Args, Parser, Subcommand}; +use datafusion::{datasource::MemTable, prelude::DataFrame}; +use datafusion_common::DataFusionError; +use datafusion_expr::{cast, col, lit, random}; +use deltalake_core::protocol::SaveMode; +use deltalake_core::{ + arrow::{ + self, + datatypes::{DataType, Field}, + }, + datafusion::prelude::{CsvReadOptions, SessionContext}, + delta_datafusion::{DeltaScanConfig, DeltaTableProvider}, + operations::merge::{MergeBuilder, MergeMetrics}, + DeltaOps, DeltaTable, DeltaTableBuilder, DeltaTableError, ObjectStore, Path, +}; +use serde_json::json; +use tokio::time::Instant; + +/* Convert web_returns dataset from TPC DS's datagen utility into a Delta table + This table will be partitioned on `wr_returned_date_sk` +*/ +pub async fn convert_tpcds_web_returns(input_path: String, table_path: String) -> Result<(), ()> { + let ctx = SessionContext::new(); + + let schema = ArrowSchema::new(vec![ + Field::new("wr_returned_date_sk", DataType::Int64, true), + Field::new("wr_returned_time_sk", DataType::Int64, true), + Field::new("wr_item_sk", DataType::Int64, false), + Field::new("wr_refunded_customer_sk", DataType::Int64, true), + Field::new("wr_refunded_cdemo_sk", DataType::Int64, true), + Field::new("wr_refunded_hdemo_sk", DataType::Int64, true), + Field::new("wr_refunded_addr_sk", DataType::Int64, true), + Field::new("wr_returning_customer_sk", DataType::Int64, true), + Field::new("wr_returning_cdemo_sk", DataType::Int64, true), + Field::new("wr_returning_hdemo_sk", DataType::Int64, true), + Field::new("wr_returning_addr_sk", DataType::Int64, true), + Field::new("wr_web_page_sk", DataType::Int64, true), + Field::new("wr_reason_sk", DataType::Int64, true), + Field::new("wr_order_number", DataType::Int64, false), + Field::new("wr_return_quantity", DataType::Int32, true), + Field::new("wr_return_amt", DataType::Decimal128(7, 2), true), + Field::new("wr_return_tax", DataType::Decimal128(7, 2), true), + Field::new("wr_return_amt_inc_tax", DataType::Decimal128(7, 2), true), + Field::new("wr_fee", DataType::Decimal128(7, 2), true), + Field::new("wr_return_ship_cost", DataType::Decimal128(7, 2), true), + Field::new("wr_refunded_cash", DataType::Decimal128(7, 2), true), + Field::new("wr_reversed_charge", DataType::Decimal128(7, 2), true), + Field::new("wr_account_credit", DataType::Decimal128(7, 2), true), + Field::new("wr_net_loss", DataType::Decimal128(7, 2), true), + ]); + + let table = ctx + .read_csv( + input_path, + CsvReadOptions { + has_header: false, + delimiter: b'|', + file_extension: ".dat", + schema: Some(&schema), + ..Default::default() + }, + ) + .await + .unwrap(); + + DeltaOps::try_from_uri(table_path) + .await + .unwrap() + .write(table.collect().await.unwrap()) + .with_partition_columns(vec!["wr_returned_date_sk"]) + .await + .unwrap(); + + Ok(()) +} + +fn merge_upsert(source: DataFrame, table: DeltaTable) -> Result { + DeltaOps(table) + .merge(source, "source.wr_item_sk = target.wr_item_sk and source.wr_order_number = target.wr_order_number") + .with_source_alias("source") + .with_target_alias("target") + .when_matched_update(|update| { + update + .update("wr_returned_date_sk", "source.wr_returned_date_sk") + .update("wr_returned_time_sk", "source.wr_returned_time_sk") + .update("wr_item_sk", "source.wr_item_sk") + .update("wr_refunded_customer_sk", "source.wr_refunded_customer_sk") + .update("wr_refunded_cdemo_sk", "source.wr_refunded_cdemo_sk") + .update("wr_refunded_hdemo_sk", "source.wr_refunded_hdemo_sk") + .update("wr_refunded_addr_sk", "source.wr_refunded_addr_sk") + .update("wr_returning_customer_sk", "source.wr_returning_customer_sk") + .update("wr_returning_cdemo_sk", "source.wr_returning_cdemo_sk") + .update("wr_returning_hdemo_sk", "source.wr_returning_hdemo_sk") + .update("wr_returning_addr_sk", "source.wr_returning_addr_sk") + .update("wr_web_page_sk", "source.wr_web_page_sk") + .update("wr_reason_sk", "source.wr_reason_sk") + .update("wr_order_number", "source.wr_order_number") + .update("wr_return_quantity", "source.wr_return_quantity") + .update("wr_return_amt", "source.wr_return_amt") + .update("wr_return_tax", "source.wr_return_tax") + .update("wr_return_amt_inc_tax", "source.wr_return_amt_inc_tax") + .update("wr_fee", "source.wr_fee") + .update("wr_return_ship_cost", "source.wr_return_ship_cost") + .update("wr_refunded_cash", "source.wr_refunded_cash") + .update("wr_reversed_charge", "source.wr_reversed_charge") + .update("wr_account_credit", "source.wr_account_credit") + .update("wr_net_loss", "source.wr_net_loss") + })? + .when_not_matched_insert(|insert| { + insert + .set("wr_returned_date_sk", "source.wr_returned_date_sk") + .set("wr_returned_time_sk", "source.wr_returned_time_sk") + .set("wr_item_sk", "source.wr_item_sk") + .set("wr_refunded_customer_sk", "source.wr_refunded_customer_sk") + .set("wr_refunded_cdemo_sk", "source.wr_refunded_cdemo_sk") + .set("wr_refunded_hdemo_sk", "source.wr_refunded_hdemo_sk") + .set("wr_refunded_addr_sk", "source.wr_refunded_addr_sk") + .set("wr_returning_customer_sk", "source.wr_returning_customer_sk") + .set("wr_returning_cdemo_sk", "source.wr_returning_cdemo_sk") + .set("wr_returning_hdemo_sk", "source.wr_returning_hdemo_sk") + .set("wr_returning_addr_sk", "source.wr_returning_addr_sk") + .set("wr_web_page_sk", "source.wr_web_page_sk") + .set("wr_reason_sk", "source.wr_reason_sk") + .set("wr_order_number", "source.wr_order_number") + .set("wr_return_quantity", "source.wr_return_quantity") + .set("wr_return_amt", "source.wr_return_amt") + .set("wr_return_tax", "source.wr_return_tax") + .set("wr_return_amt_inc_tax", "source.wr_return_amt_inc_tax") + .set("wr_fee", "source.wr_fee") + .set("wr_return_ship_cost", "source.wr_return_ship_cost") + .set("wr_refunded_cash", "source.wr_refunded_cash") + .set("wr_reversed_charge", "source.wr_reversed_charge") + .set("wr_account_credit", "source.wr_account_credit") + .set("wr_net_loss", "source.wr_net_loss") + }) +} + +fn merge_insert(source: DataFrame, table: DeltaTable) -> Result { + DeltaOps(table) + .merge(source, "source.wr_item_sk = target.wr_item_sk and source.wr_order_number = target.wr_order_number") + .with_source_alias("source") + .with_target_alias("target") + .when_not_matched_insert(|insert| { + insert + .set("wr_returned_date_sk", "source.wr_returned_date_sk") + .set("wr_returned_time_sk", "source.wr_returned_time_sk") + .set("wr_item_sk", "source.wr_item_sk") + .set("wr_refunded_customer_sk", "source.wr_refunded_customer_sk") + .set("wr_refunded_cdemo_sk", "source.wr_refunded_cdemo_sk") + .set("wr_refunded_hdemo_sk", "source.wr_refunded_hdemo_sk") + .set("wr_refunded_addr_sk", "source.wr_refunded_addr_sk") + .set("wr_returning_customer_sk", "source.wr_returning_customer_sk") + .set("wr_returning_cdemo_sk", "source.wr_returning_cdemo_sk") + .set("wr_returning_hdemo_sk", "source.wr_returning_hdemo_sk") + .set("wr_returning_addr_sk", "source.wr_returning_addr_sk") + .set("wr_web_page_sk", "source.wr_web_page_sk") + .set("wr_reason_sk", "source.wr_reason_sk") + .set("wr_order_number", "source.wr_order_number") + .set("wr_return_quantity", "source.wr_return_quantity") + .set("wr_return_amt", "source.wr_return_amt") + .set("wr_return_tax", "source.wr_return_tax") + .set("wr_return_amt_inc_tax", "source.wr_return_amt_inc_tax") + .set("wr_fee", "source.wr_fee") + .set("wr_return_ship_cost", "source.wr_return_ship_cost") + .set("wr_refunded_cash", "source.wr_refunded_cash") + .set("wr_reversed_charge", "source.wr_reversed_charge") + .set("wr_account_credit", "source.wr_account_credit") + .set("wr_net_loss", "source.wr_net_loss") + }) +} + +fn merge_delete(source: DataFrame, table: DeltaTable) -> Result { + DeltaOps(table) + .merge(source, "source.wr_item_sk = target.wr_item_sk and source.wr_order_number = target.wr_order_number") + .with_source_alias("source") + .with_target_alias("target") + .when_matched_delete(|delete| { + delete + }) +} + +async fn benchmark_merge_tpcds( + path: String, + parameters: MergePerfParams, + merge: fn(DataFrame, DeltaTable) -> Result, +) -> Result<(core::time::Duration, MergeMetrics), DataFusionError> { + let table = DeltaTableBuilder::from_uri(path).load().await?; + let file_count = table.state.files().len(); + + let provider = DeltaTableProvider::try_new( + table.state.clone(), + table.log_store(), + DeltaScanConfig { + file_column_name: Some("file_path".to_string()), + }, + ) + .unwrap(); + + let ctx = SessionContext::new(); + ctx.register_table("t1", Arc::new(provider))?; + + let files = ctx + .sql("select file_path as file from t1 group by file") + .await? + .with_column("r", random())? + .filter(col("r").lt_eq(lit(parameters.sample_files)))?; + + let file_sample = files.collect_partitioned().await?; + let schema = file_sample.get(0).unwrap().get(0).unwrap().schema(); + let mem_table = Arc::new(MemTable::try_new(schema, file_sample)?); + ctx.register_table("file_sample", mem_table)?; + let file_sample_count = ctx.table("file_sample").await?.count().await?; + + let row_sample = ctx.table("t1").await?.join( + ctx.table("file_sample").await?, + datafusion_common::JoinType::Inner, + &["file_path"], + &["file"], + None, + )?; + + let matched = row_sample + .clone() + .filter(random().lt_eq(lit(parameters.sample_matched_rows)))?; + + let rand = cast(random() * lit(u32::MAX), DataType::Int64); + let not_matched = row_sample + .filter(random().lt_eq(lit(parameters.sample_not_matched_rows)))? + .with_column("wr_item_sk", rand.clone())? + .with_column("wr_order_number", rand)?; + + let source = matched.union(not_matched)?; + + let start = Instant::now(); + let (table, metrics) = merge(source, table)?.await?; + let end = Instant::now(); + + let duration = end.duration_since(start); + + println!("Total File count: {}", file_count); + println!("File sample count: {}", file_sample_count); + println!("{:?}", metrics); + println!("Seconds: {}", duration.as_secs_f32()); + + // Clean up and restore to original state. + let (table, _) = DeltaOps(table).restore().with_version_to_restore(0).await?; + let (table, _) = DeltaOps(table) + .vacuum() + .with_retention_period(Duration::seconds(0)) + .with_enforce_retention_duration(false) + .await?; + table + .object_store() + .delete(&Path::parse("_delta_log/00000000000000000001.json")?) + .await?; + table + .object_store() + .delete(&Path::parse("_delta_log/00000000000000000002.json")?) + .await?; + + Ok((duration, metrics)) +} + +#[derive(Subcommand, Debug)] +enum Command { + Convert(Convert), + Bench(BenchArg), + Standard(Standard), + Compare(Compare), + Show(Show), +} + +#[derive(Debug, Args)] +struct Convert { + tpcds_path: String, + delta_path: String, +} + +#[derive(Debug, Args)] +struct Standard { + delta_path: String, + samples: Option, + output_path: Option, + group_id: Option, +} + +#[derive(Debug, Args)] +struct Compare { + before_path: String, + before_group_id: String, + after_path: String, + after_group_id: String, +} + +#[derive(Debug, Args)] +struct Show { + path: String, +} + +#[derive(Debug, Args)] +struct BenchArg { + table_path: String, + #[command(subcommand)] + name: MergeBench, +} + +struct Bench { + name: String, + op: fn(DataFrame, DeltaTable) -> Result, + params: MergePerfParams, +} + +impl Bench { + fn new( + name: S, + op: fn(DataFrame, DeltaTable) -> Result, + params: MergePerfParams, + ) -> Self { + Bench { + name: name.to_string(), + op, + params, + } + } +} + +#[derive(Debug, Args, Clone)] +struct MergePerfParams { + pub sample_files: f32, + pub sample_matched_rows: f32, + pub sample_not_matched_rows: f32, +} + +#[derive(Debug, Clone, Subcommand)] +enum MergeBench { + Upsert(MergePerfParams), + Delete(MergePerfParams), + Insert(MergePerfParams), +} + +#[derive(Parser, Debug)] +#[command(about)] +struct MergePrefArgs { + #[command(subcommand)] + command: Command, +} + +#[tokio::main] +async fn main() { + match MergePrefArgs::parse().command { + Command::Convert(Convert { + tpcds_path, + delta_path, + }) => { + convert_tpcds_web_returns(tpcds_path, delta_path) + .await + .unwrap(); + } + Command::Bench(BenchArg { table_path, name }) => { + let (merge_op, params): ( + fn(DataFrame, DeltaTable) -> Result, + MergePerfParams, + ) = match name { + MergeBench::Upsert(params) => (merge_upsert, params), + MergeBench::Delete(params) => (merge_delete, params), + MergeBench::Insert(params) => (merge_insert, params), + }; + + benchmark_merge_tpcds(table_path, params, merge_op) + .await + .unwrap(); + } + Command::Standard(Standard { + delta_path, + samples, + output_path, + group_id, + }) => { + let benches = vec![Bench::new( + "delete_only_fileMatchedFraction_0.05_rowMatchedFraction_0.05", + merge_delete, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.05, + sample_not_matched_rows: 0.0, + }, + ), + Bench::new( + "multiple_insert_only_fileMatchedFraction_0.05_rowNotMatchedFraction_0.05", + merge_insert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.00, + sample_not_matched_rows: 0.05, + }, + ), + Bench::new( + "multiple_insert_only_fileMatchedFraction_0.05_rowNotMatchedFraction_0.50", + merge_insert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.00, + sample_not_matched_rows: 0.50, + }, + ), + Bench::new( + "multiple_insert_only_fileMatchedFraction_0.05_rowNotMatchedFraction_1.0", + merge_insert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.00, + sample_not_matched_rows: 1.0, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.01_rowNotMatchedFraction_0.1", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.01, + sample_not_matched_rows: 0.1, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.0_rowNotMatchedFraction_0.1", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.00, + sample_not_matched_rows: 0.1, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.1_rowNotMatchedFraction_0.0", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.1, + sample_not_matched_rows: 0.0, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.1_rowNotMatchedFraction_0.01", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.1, + sample_not_matched_rows: 0.01, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.5_rowNotMatchedFraction_0.001", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.5, + sample_not_matched_rows: 0.001, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.99_rowNotMatchedFraction_0.001", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 0.99, + sample_not_matched_rows: 0.001, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.05_rowMatchedFraction_1.0_rowNotMatchedFraction_0.001", + merge_upsert, + MergePerfParams { + sample_files: 0.05, + sample_matched_rows: 1.0, + sample_not_matched_rows: 0.001, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_0.5_rowMatchedFraction_0.001_rowNotMatchedFraction_0.001", + merge_upsert, + MergePerfParams { + sample_files: 0.5, + sample_matched_rows: 0.001, + sample_not_matched_rows: 0.001, + }, + ), + Bench::new( + "upsert_fileMatchedFraction_1.0_rowMatchedFraction_0.001_rowNotMatchedFraction_0.001", + merge_upsert, + MergePerfParams { + sample_files: 1.0, + sample_matched_rows: 0.001, + sample_not_matched_rows: 0.001, + }, + ) + ]; + + let num_samples = samples.unwrap_or(1); + let group_id = group_id.unwrap_or( + SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() + .to_string(), + ); + let output = output_path.unwrap_or("data/benchmarks".into()); + + let mut group_ids = vec![]; + let mut name = vec![]; + let mut samples = vec![]; + let mut duration_ms = vec![]; + let mut data = vec![]; + + for bench in benches { + for sample in 0..num_samples { + println!("Test: {} Sample: {}", bench.name, sample); + let res = + benchmark_merge_tpcds(delta_path.clone(), bench.params.clone(), bench.op) + .await + .unwrap(); + + group_ids.push(group_id.clone()); + name.push(bench.name.clone()); + samples.push(sample); + duration_ms.push(res.0.as_millis() as u32); + data.push(json!(res.1).to_string()); + } + } + + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("group_id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, false), + Field::new("sample", DataType::UInt32, false), + Field::new("duration_ms", DataType::UInt32, false), + Field::new("data", DataType::Utf8, true), + ])); + + let batch = RecordBatch::try_new( + schema, + vec![ + Arc::new(StringArray::from(group_ids)), + Arc::new(StringArray::from(name)), + Arc::new(UInt32Array::from(samples)), + Arc::new(UInt32Array::from(duration_ms)), + Arc::new(StringArray::from(data)), + ], + ) + .unwrap(); + + DeltaOps::try_from_uri(output) + .await + .unwrap() + .write(vec![batch]) + .with_save_mode(SaveMode::Append) + .await + .unwrap(); + } + Command::Compare(Compare { + before_path, + before_group_id, + after_path, + after_group_id, + }) => { + let before_table = DeltaTableBuilder::from_uri(before_path) + .load() + .await + .unwrap(); + let after_table = DeltaTableBuilder::from_uri(after_path) + .load() + .await + .unwrap(); + + let ctx = SessionContext::new(); + ctx.register_table("before", Arc::new(before_table)) + .unwrap(); + ctx.register_table("after", Arc::new(after_table)).unwrap(); + + let before_stats = ctx + .sql(&format!( + " + select name as before_name, + avg(cast(duration_ms as float)) as before_duration_avg + from before where group_id = {} + group by name + ", + before_group_id + )) + .await + .unwrap(); + + let after_stats = ctx + .sql(&format!( + " + select name as after_name, + avg(cast(duration_ms as float)) as after_duration_avg + from after where group_id = {} + group by name + ", + after_group_id + )) + .await + .unwrap(); + + before_stats + .join( + after_stats, + datafusion_common::JoinType::Inner, + &["before_name"], + &["after_name"], + None, + ) + .unwrap() + .select(vec![ + col("before_name").alias("name"), + col("before_duration_avg"), + col("after_duration_avg"), + (col("before_duration_avg") / (col("after_duration_avg"))), + ]) + .unwrap() + .sort(vec![col("name").sort(true, true)]) + .unwrap() + .show() + .await + .unwrap(); + } + Command::Show(Show { path }) => { + let stats = DeltaTableBuilder::from_uri(path).load().await.unwrap(); + let ctx = SessionContext::new(); + ctx.register_table("stats", Arc::new(stats)).unwrap(); + + ctx.sql("select * from stats") + .await + .unwrap() + .show() + .await + .unwrap(); + } + } +} From 07113c6badd33cc7e363fa3e75272e2ea1d55834 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Mon, 20 Nov 2023 21:49:47 +0100 Subject: [PATCH 18/35] Revert "refactor: express log schema in delta types" This reverts commit 57565b57db0e05ae79524d6e5d1bbc437e614015. --- crates/benchmarks/Cargo.toml | 46 - crates/benchmarks/README.md | 55 - crates/benchmarks/src/bin/merge.rs | 647 ---------- .../src/delta_datafusion/logical.rs | 48 - .../src/delta_datafusion/mod.rs | 9 +- .../src/delta_datafusion/physical.rs | 180 --- .../kernel/{arrow/mod.rs => actions/arrow.rs} | 17 +- .../deltalake-core/src/kernel/actions/mod.rs | 10 +- .../src/kernel/actions/schemas.rs | 481 ++++---- .../src/kernel/actions/types.rs | 125 +- .../src/kernel/arrow/schemas.rs | 63 - crates/deltalake-core/src/kernel/error.rs | 3 +- crates/deltalake-core/src/kernel/mod.rs | 2 - .../src/operations/convert_to_delta.rs | 16 +- crates/deltalake-core/src/operations/merge.rs | 631 +++++----- crates/deltalake-core/src/operations/mod.rs | 136 ++- .../deltalake-core/src/operations/update.rs | 5 +- .../src/protocol/checkpoints.rs | 2 +- crates/deltalake-core/src/protocol/mod.rs | 20 +- .../src/schema/arrow_convert.rs | 1049 +++++++++++++++++ crates/deltalake-core/src/table/config.rs | 56 - python/deltalake/__init__.py | 1 - python/deltalake/_internal.pyi | 10 - python/deltalake/writer.py | 55 - python/src/lib.rs | 68 +- python/tests/test_convert_to_delta.py | 97 -- 26 files changed, 1828 insertions(+), 2004 deletions(-) delete mode 100644 crates/benchmarks/Cargo.toml delete mode 100644 crates/benchmarks/README.md delete mode 100644 crates/benchmarks/src/bin/merge.rs delete mode 100644 crates/deltalake-core/src/delta_datafusion/logical.rs delete mode 100644 crates/deltalake-core/src/delta_datafusion/physical.rs rename crates/deltalake-core/src/kernel/{arrow/mod.rs => actions/arrow.rs} (98%) delete mode 100644 crates/deltalake-core/src/kernel/arrow/schemas.rs create mode 100644 crates/deltalake-core/src/schema/arrow_convert.rs delete mode 100644 python/tests/test_convert_to_delta.py diff --git a/crates/benchmarks/Cargo.toml b/crates/benchmarks/Cargo.toml deleted file mode 100644 index 76bcc8a312..0000000000 --- a/crates/benchmarks/Cargo.toml +++ /dev/null @@ -1,46 +0,0 @@ -[package] -name = "delta-benchmarks" -version = "0.0.1" -authors = ["David Blajda "] -homepage = "https://github.com/delta-io/delta.rs" -license = "Apache-2.0" -keywords = ["deltalake", "delta", "datalake"] -description = "Delta-rs Benchmarks" -edition = "2021" - -[dependencies] -clap = { version = "4", features = [ "derive" ] } -chrono = { version = "0.4.31", default-features = false, features = ["clock"] } -tokio = { version = "1", features = ["fs", "macros", "rt", "io-util"] } -env_logger = "0" - -# arrow -arrow = { workspace = true } -arrow-array = { workspace = true } -arrow-buffer = { workspace = true } -arrow-cast = { workspace = true } -arrow-ord = { workspace = true } -arrow-row = { workspace = true } -arrow-schema = { workspace = true, features = ["serde"] } -arrow-select = { workspace = true } -parquet = { workspace = true, features = [ - "async", - "object_store", -] } - -# serde -serde = { workspace = true, features = ["derive"] } -serde_json = { workspace = true } - -# datafusion -datafusion = { workspace = true } -datafusion-expr = { workspace = true } -datafusion-common = { workspace = true } -datafusion-proto = { workspace = true } -datafusion-sql = { workspace = true } -datafusion-physical-expr = { workspace = true } - -[dependencies.deltalake-core] -path = "../deltalake-core" -version = "0" -features = ["datafusion"] diff --git a/crates/benchmarks/README.md b/crates/benchmarks/README.md deleted file mode 100644 index c5d6b0b920..0000000000 --- a/crates/benchmarks/README.md +++ /dev/null @@ -1,55 +0,0 @@ -# Merge -The merge benchmarks are similar to the ones used by [Delta Spark](https://github.com/delta-io/delta/pull/1835). - - -## Dataset - -Databricks maintains a public S3 bucket of the TPC-DS dataset with various factor where requesters must pay to download this dataset. Below is an example of how to list the 1gb scale factor - -``` -aws s3api list-objects --bucket devrel-delta-datasets --request-payer requester --prefix tpcds-2.13/tpcds_sf1_parquet/web_returns/ -``` - -You can generate the TPC-DS dataset yourself by downloading and compiling [the generator](https://www.tpc.org/tpc_documents_current_versions/current_specifications5.asp) -You may need to update the CFLAGS to include `-fcommon` to compile on newer versions of GCC. - -## Commands -These commands can be executed from the root of the benchmark crate. Some commands depend on the existance of the TPC-DS Dataset existing. - -### Convert -Converts a TPC-DS web_returns csv into a Delta table -Assumes the dataset is pipe delimited and records do not have a trailing delimiter - -``` - cargo run --release --bin merge -- convert data/tpcds/web_returns.dat data/web_returns -``` - -### Standard -Execute the standard merge bench suite. -Results can be saved to a delta table for further analysis. -This table has the following schema: - -group_id: Used to group all tests that executed as a part of this call. Default value is the timestamp of execution -name: The benchmark name that was executed -sample: The iteration number for a given benchmark name -duration_ms: How long the benchmark took in ms -data: Free field to pack any additonal data - -``` - cargo run --release --bin merge -- standard data/web_returns 1 data/merge_results -``` - -### Compare -Compare the results of two different runs. -The a Delta table paths and the `group_id` of each run and obtain the speedup for each test case - -``` - cargo run --release --bin merge -- compare data/benchmarks/ 1698636172801 data/benchmarks/ 1699759539902 -``` - -### Show -Show all benchmarks results from a delta table - -``` - cargo run --release --bin merge -- show data/benchmark -``` diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs deleted file mode 100644 index 5afa3e6f35..0000000000 --- a/crates/benchmarks/src/bin/merge.rs +++ /dev/null @@ -1,647 +0,0 @@ -use std::{ - sync::Arc, - time::{SystemTime, UNIX_EPOCH}, -}; - -use arrow::datatypes::Schema as ArrowSchema; -use arrow_array::{RecordBatch, StringArray, UInt32Array}; -use chrono::Duration; -use clap::{command, Args, Parser, Subcommand}; -use datafusion::{datasource::MemTable, prelude::DataFrame}; -use datafusion_common::DataFusionError; -use datafusion_expr::{cast, col, lit, random}; -use deltalake_core::protocol::SaveMode; -use deltalake_core::{ - arrow::{ - self, - datatypes::{DataType, Field}, - }, - datafusion::prelude::{CsvReadOptions, SessionContext}, - delta_datafusion::{DeltaScanConfig, DeltaTableProvider}, - operations::merge::{MergeBuilder, MergeMetrics}, - DeltaOps, DeltaTable, DeltaTableBuilder, DeltaTableError, ObjectStore, Path, -}; -use serde_json::json; -use tokio::time::Instant; - -/* Convert web_returns dataset from TPC DS's datagen utility into a Delta table - This table will be partitioned on `wr_returned_date_sk` -*/ -pub async fn convert_tpcds_web_returns(input_path: String, table_path: String) -> Result<(), ()> { - let ctx = SessionContext::new(); - - let schema = ArrowSchema::new(vec![ - Field::new("wr_returned_date_sk", DataType::Int64, true), - Field::new("wr_returned_time_sk", DataType::Int64, true), - Field::new("wr_item_sk", DataType::Int64, false), - Field::new("wr_refunded_customer_sk", DataType::Int64, true), - Field::new("wr_refunded_cdemo_sk", DataType::Int64, true), - Field::new("wr_refunded_hdemo_sk", DataType::Int64, true), - Field::new("wr_refunded_addr_sk", DataType::Int64, true), - Field::new("wr_returning_customer_sk", DataType::Int64, true), - Field::new("wr_returning_cdemo_sk", DataType::Int64, true), - Field::new("wr_returning_hdemo_sk", DataType::Int64, true), - Field::new("wr_returning_addr_sk", DataType::Int64, true), - Field::new("wr_web_page_sk", DataType::Int64, true), - Field::new("wr_reason_sk", DataType::Int64, true), - Field::new("wr_order_number", DataType::Int64, false), - Field::new("wr_return_quantity", DataType::Int32, true), - Field::new("wr_return_amt", DataType::Decimal128(7, 2), true), - Field::new("wr_return_tax", DataType::Decimal128(7, 2), true), - Field::new("wr_return_amt_inc_tax", DataType::Decimal128(7, 2), true), - Field::new("wr_fee", DataType::Decimal128(7, 2), true), - Field::new("wr_return_ship_cost", DataType::Decimal128(7, 2), true), - Field::new("wr_refunded_cash", DataType::Decimal128(7, 2), true), - Field::new("wr_reversed_charge", DataType::Decimal128(7, 2), true), - Field::new("wr_account_credit", DataType::Decimal128(7, 2), true), - Field::new("wr_net_loss", DataType::Decimal128(7, 2), true), - ]); - - let table = ctx - .read_csv( - input_path, - CsvReadOptions { - has_header: false, - delimiter: b'|', - file_extension: ".dat", - schema: Some(&schema), - ..Default::default() - }, - ) - .await - .unwrap(); - - DeltaOps::try_from_uri(table_path) - .await - .unwrap() - .write(table.collect().await.unwrap()) - .with_partition_columns(vec!["wr_returned_date_sk"]) - .await - .unwrap(); - - Ok(()) -} - -fn merge_upsert(source: DataFrame, table: DeltaTable) -> Result { - DeltaOps(table) - .merge(source, "source.wr_item_sk = target.wr_item_sk and source.wr_order_number = target.wr_order_number") - .with_source_alias("source") - .with_target_alias("target") - .when_matched_update(|update| { - update - .update("wr_returned_date_sk", "source.wr_returned_date_sk") - .update("wr_returned_time_sk", "source.wr_returned_time_sk") - .update("wr_item_sk", "source.wr_item_sk") - .update("wr_refunded_customer_sk", "source.wr_refunded_customer_sk") - .update("wr_refunded_cdemo_sk", "source.wr_refunded_cdemo_sk") - .update("wr_refunded_hdemo_sk", "source.wr_refunded_hdemo_sk") - .update("wr_refunded_addr_sk", "source.wr_refunded_addr_sk") - .update("wr_returning_customer_sk", "source.wr_returning_customer_sk") - .update("wr_returning_cdemo_sk", "source.wr_returning_cdemo_sk") - .update("wr_returning_hdemo_sk", "source.wr_returning_hdemo_sk") - .update("wr_returning_addr_sk", "source.wr_returning_addr_sk") - .update("wr_web_page_sk", "source.wr_web_page_sk") - .update("wr_reason_sk", "source.wr_reason_sk") - .update("wr_order_number", "source.wr_order_number") - .update("wr_return_quantity", "source.wr_return_quantity") - .update("wr_return_amt", "source.wr_return_amt") - .update("wr_return_tax", "source.wr_return_tax") - .update("wr_return_amt_inc_tax", "source.wr_return_amt_inc_tax") - .update("wr_fee", "source.wr_fee") - .update("wr_return_ship_cost", "source.wr_return_ship_cost") - .update("wr_refunded_cash", "source.wr_refunded_cash") - .update("wr_reversed_charge", "source.wr_reversed_charge") - .update("wr_account_credit", "source.wr_account_credit") - .update("wr_net_loss", "source.wr_net_loss") - })? - .when_not_matched_insert(|insert| { - insert - .set("wr_returned_date_sk", "source.wr_returned_date_sk") - .set("wr_returned_time_sk", "source.wr_returned_time_sk") - .set("wr_item_sk", "source.wr_item_sk") - .set("wr_refunded_customer_sk", "source.wr_refunded_customer_sk") - .set("wr_refunded_cdemo_sk", "source.wr_refunded_cdemo_sk") - .set("wr_refunded_hdemo_sk", "source.wr_refunded_hdemo_sk") - .set("wr_refunded_addr_sk", "source.wr_refunded_addr_sk") - .set("wr_returning_customer_sk", "source.wr_returning_customer_sk") - .set("wr_returning_cdemo_sk", "source.wr_returning_cdemo_sk") - .set("wr_returning_hdemo_sk", "source.wr_returning_hdemo_sk") - .set("wr_returning_addr_sk", "source.wr_returning_addr_sk") - .set("wr_web_page_sk", "source.wr_web_page_sk") - .set("wr_reason_sk", "source.wr_reason_sk") - .set("wr_order_number", "source.wr_order_number") - .set("wr_return_quantity", "source.wr_return_quantity") - .set("wr_return_amt", "source.wr_return_amt") - .set("wr_return_tax", "source.wr_return_tax") - .set("wr_return_amt_inc_tax", "source.wr_return_amt_inc_tax") - .set("wr_fee", "source.wr_fee") - .set("wr_return_ship_cost", "source.wr_return_ship_cost") - .set("wr_refunded_cash", "source.wr_refunded_cash") - .set("wr_reversed_charge", "source.wr_reversed_charge") - .set("wr_account_credit", "source.wr_account_credit") - .set("wr_net_loss", "source.wr_net_loss") - }) -} - -fn merge_insert(source: DataFrame, table: DeltaTable) -> Result { - DeltaOps(table) - .merge(source, "source.wr_item_sk = target.wr_item_sk and source.wr_order_number = target.wr_order_number") - .with_source_alias("source") - .with_target_alias("target") - .when_not_matched_insert(|insert| { - insert - .set("wr_returned_date_sk", "source.wr_returned_date_sk") - .set("wr_returned_time_sk", "source.wr_returned_time_sk") - .set("wr_item_sk", "source.wr_item_sk") - .set("wr_refunded_customer_sk", "source.wr_refunded_customer_sk") - .set("wr_refunded_cdemo_sk", "source.wr_refunded_cdemo_sk") - .set("wr_refunded_hdemo_sk", "source.wr_refunded_hdemo_sk") - .set("wr_refunded_addr_sk", "source.wr_refunded_addr_sk") - .set("wr_returning_customer_sk", "source.wr_returning_customer_sk") - .set("wr_returning_cdemo_sk", "source.wr_returning_cdemo_sk") - .set("wr_returning_hdemo_sk", "source.wr_returning_hdemo_sk") - .set("wr_returning_addr_sk", "source.wr_returning_addr_sk") - .set("wr_web_page_sk", "source.wr_web_page_sk") - .set("wr_reason_sk", "source.wr_reason_sk") - .set("wr_order_number", "source.wr_order_number") - .set("wr_return_quantity", "source.wr_return_quantity") - .set("wr_return_amt", "source.wr_return_amt") - .set("wr_return_tax", "source.wr_return_tax") - .set("wr_return_amt_inc_tax", "source.wr_return_amt_inc_tax") - .set("wr_fee", "source.wr_fee") - .set("wr_return_ship_cost", "source.wr_return_ship_cost") - .set("wr_refunded_cash", "source.wr_refunded_cash") - .set("wr_reversed_charge", "source.wr_reversed_charge") - .set("wr_account_credit", "source.wr_account_credit") - .set("wr_net_loss", "source.wr_net_loss") - }) -} - -fn merge_delete(source: DataFrame, table: DeltaTable) -> Result { - DeltaOps(table) - .merge(source, "source.wr_item_sk = target.wr_item_sk and source.wr_order_number = target.wr_order_number") - .with_source_alias("source") - .with_target_alias("target") - .when_matched_delete(|delete| { - delete - }) -} - -async fn benchmark_merge_tpcds( - path: String, - parameters: MergePerfParams, - merge: fn(DataFrame, DeltaTable) -> Result, -) -> Result<(core::time::Duration, MergeMetrics), DataFusionError> { - let table = DeltaTableBuilder::from_uri(path).load().await?; - let file_count = table.state.files().len(); - - let provider = DeltaTableProvider::try_new( - table.state.clone(), - table.log_store(), - DeltaScanConfig { - file_column_name: Some("file_path".to_string()), - }, - ) - .unwrap(); - - let ctx = SessionContext::new(); - ctx.register_table("t1", Arc::new(provider))?; - - let files = ctx - .sql("select file_path as file from t1 group by file") - .await? - .with_column("r", random())? - .filter(col("r").lt_eq(lit(parameters.sample_files)))?; - - let file_sample = files.collect_partitioned().await?; - let schema = file_sample.get(0).unwrap().get(0).unwrap().schema(); - let mem_table = Arc::new(MemTable::try_new(schema, file_sample)?); - ctx.register_table("file_sample", mem_table)?; - let file_sample_count = ctx.table("file_sample").await?.count().await?; - - let row_sample = ctx.table("t1").await?.join( - ctx.table("file_sample").await?, - datafusion_common::JoinType::Inner, - &["file_path"], - &["file"], - None, - )?; - - let matched = row_sample - .clone() - .filter(random().lt_eq(lit(parameters.sample_matched_rows)))?; - - let rand = cast(random() * lit(u32::MAX), DataType::Int64); - let not_matched = row_sample - .filter(random().lt_eq(lit(parameters.sample_not_matched_rows)))? - .with_column("wr_item_sk", rand.clone())? - .with_column("wr_order_number", rand)?; - - let source = matched.union(not_matched)?; - - let start = Instant::now(); - let (table, metrics) = merge(source, table)?.await?; - let end = Instant::now(); - - let duration = end.duration_since(start); - - println!("Total File count: {}", file_count); - println!("File sample count: {}", file_sample_count); - println!("{:?}", metrics); - println!("Seconds: {}", duration.as_secs_f32()); - - // Clean up and restore to original state. - let (table, _) = DeltaOps(table).restore().with_version_to_restore(0).await?; - let (table, _) = DeltaOps(table) - .vacuum() - .with_retention_period(Duration::seconds(0)) - .with_enforce_retention_duration(false) - .await?; - table - .object_store() - .delete(&Path::parse("_delta_log/00000000000000000001.json")?) - .await?; - table - .object_store() - .delete(&Path::parse("_delta_log/00000000000000000002.json")?) - .await?; - - Ok((duration, metrics)) -} - -#[derive(Subcommand, Debug)] -enum Command { - Convert(Convert), - Bench(BenchArg), - Standard(Standard), - Compare(Compare), - Show(Show), -} - -#[derive(Debug, Args)] -struct Convert { - tpcds_path: String, - delta_path: String, -} - -#[derive(Debug, Args)] -struct Standard { - delta_path: String, - samples: Option, - output_path: Option, - group_id: Option, -} - -#[derive(Debug, Args)] -struct Compare { - before_path: String, - before_group_id: String, - after_path: String, - after_group_id: String, -} - -#[derive(Debug, Args)] -struct Show { - path: String, -} - -#[derive(Debug, Args)] -struct BenchArg { - table_path: String, - #[command(subcommand)] - name: MergeBench, -} - -struct Bench { - name: String, - op: fn(DataFrame, DeltaTable) -> Result, - params: MergePerfParams, -} - -impl Bench { - fn new( - name: S, - op: fn(DataFrame, DeltaTable) -> Result, - params: MergePerfParams, - ) -> Self { - Bench { - name: name.to_string(), - op, - params, - } - } -} - -#[derive(Debug, Args, Clone)] -struct MergePerfParams { - pub sample_files: f32, - pub sample_matched_rows: f32, - pub sample_not_matched_rows: f32, -} - -#[derive(Debug, Clone, Subcommand)] -enum MergeBench { - Upsert(MergePerfParams), - Delete(MergePerfParams), - Insert(MergePerfParams), -} - -#[derive(Parser, Debug)] -#[command(about)] -struct MergePrefArgs { - #[command(subcommand)] - command: Command, -} - -#[tokio::main] -async fn main() { - match MergePrefArgs::parse().command { - Command::Convert(Convert { - tpcds_path, - delta_path, - }) => { - convert_tpcds_web_returns(tpcds_path, delta_path) - .await - .unwrap(); - } - Command::Bench(BenchArg { table_path, name }) => { - let (merge_op, params): ( - fn(DataFrame, DeltaTable) -> Result, - MergePerfParams, - ) = match name { - MergeBench::Upsert(params) => (merge_upsert, params), - MergeBench::Delete(params) => (merge_delete, params), - MergeBench::Insert(params) => (merge_insert, params), - }; - - benchmark_merge_tpcds(table_path, params, merge_op) - .await - .unwrap(); - } - Command::Standard(Standard { - delta_path, - samples, - output_path, - group_id, - }) => { - let benches = vec![Bench::new( - "delete_only_fileMatchedFraction_0.05_rowMatchedFraction_0.05", - merge_delete, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.05, - sample_not_matched_rows: 0.0, - }, - ), - Bench::new( - "multiple_insert_only_fileMatchedFraction_0.05_rowNotMatchedFraction_0.05", - merge_insert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.00, - sample_not_matched_rows: 0.05, - }, - ), - Bench::new( - "multiple_insert_only_fileMatchedFraction_0.05_rowNotMatchedFraction_0.50", - merge_insert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.00, - sample_not_matched_rows: 0.50, - }, - ), - Bench::new( - "multiple_insert_only_fileMatchedFraction_0.05_rowNotMatchedFraction_1.0", - merge_insert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.00, - sample_not_matched_rows: 1.0, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.01_rowNotMatchedFraction_0.1", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.01, - sample_not_matched_rows: 0.1, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.0_rowNotMatchedFraction_0.1", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.00, - sample_not_matched_rows: 0.1, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.1_rowNotMatchedFraction_0.0", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.1, - sample_not_matched_rows: 0.0, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.1_rowNotMatchedFraction_0.01", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.1, - sample_not_matched_rows: 0.01, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.5_rowNotMatchedFraction_0.001", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.5, - sample_not_matched_rows: 0.001, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_0.99_rowNotMatchedFraction_0.001", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 0.99, - sample_not_matched_rows: 0.001, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.05_rowMatchedFraction_1.0_rowNotMatchedFraction_0.001", - merge_upsert, - MergePerfParams { - sample_files: 0.05, - sample_matched_rows: 1.0, - sample_not_matched_rows: 0.001, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_0.5_rowMatchedFraction_0.001_rowNotMatchedFraction_0.001", - merge_upsert, - MergePerfParams { - sample_files: 0.5, - sample_matched_rows: 0.001, - sample_not_matched_rows: 0.001, - }, - ), - Bench::new( - "upsert_fileMatchedFraction_1.0_rowMatchedFraction_0.001_rowNotMatchedFraction_0.001", - merge_upsert, - MergePerfParams { - sample_files: 1.0, - sample_matched_rows: 0.001, - sample_not_matched_rows: 0.001, - }, - ) - ]; - - let num_samples = samples.unwrap_or(1); - let group_id = group_id.unwrap_or( - SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() - .to_string(), - ); - let output = output_path.unwrap_or("data/benchmarks".into()); - - let mut group_ids = vec![]; - let mut name = vec![]; - let mut samples = vec![]; - let mut duration_ms = vec![]; - let mut data = vec![]; - - for bench in benches { - for sample in 0..num_samples { - println!("Test: {} Sample: {}", bench.name, sample); - let res = - benchmark_merge_tpcds(delta_path.clone(), bench.params.clone(), bench.op) - .await - .unwrap(); - - group_ids.push(group_id.clone()); - name.push(bench.name.clone()); - samples.push(sample); - duration_ms.push(res.0.as_millis() as u32); - data.push(json!(res.1).to_string()); - } - } - - let schema = Arc::new(ArrowSchema::new(vec![ - Field::new("group_id", DataType::Utf8, false), - Field::new("name", DataType::Utf8, false), - Field::new("sample", DataType::UInt32, false), - Field::new("duration_ms", DataType::UInt32, false), - Field::new("data", DataType::Utf8, true), - ])); - - let batch = RecordBatch::try_new( - schema, - vec![ - Arc::new(StringArray::from(group_ids)), - Arc::new(StringArray::from(name)), - Arc::new(UInt32Array::from(samples)), - Arc::new(UInt32Array::from(duration_ms)), - Arc::new(StringArray::from(data)), - ], - ) - .unwrap(); - - DeltaOps::try_from_uri(output) - .await - .unwrap() - .write(vec![batch]) - .with_save_mode(SaveMode::Append) - .await - .unwrap(); - } - Command::Compare(Compare { - before_path, - before_group_id, - after_path, - after_group_id, - }) => { - let before_table = DeltaTableBuilder::from_uri(before_path) - .load() - .await - .unwrap(); - let after_table = DeltaTableBuilder::from_uri(after_path) - .load() - .await - .unwrap(); - - let ctx = SessionContext::new(); - ctx.register_table("before", Arc::new(before_table)) - .unwrap(); - ctx.register_table("after", Arc::new(after_table)).unwrap(); - - let before_stats = ctx - .sql(&format!( - " - select name as before_name, - avg(cast(duration_ms as float)) as before_duration_avg - from before where group_id = {} - group by name - ", - before_group_id - )) - .await - .unwrap(); - - let after_stats = ctx - .sql(&format!( - " - select name as after_name, - avg(cast(duration_ms as float)) as after_duration_avg - from after where group_id = {} - group by name - ", - after_group_id - )) - .await - .unwrap(); - - before_stats - .join( - after_stats, - datafusion_common::JoinType::Inner, - &["before_name"], - &["after_name"], - None, - ) - .unwrap() - .select(vec![ - col("before_name").alias("name"), - col("before_duration_avg"), - col("after_duration_avg"), - (col("before_duration_avg") / (col("after_duration_avg"))), - ]) - .unwrap() - .sort(vec![col("name").sort(true, true)]) - .unwrap() - .show() - .await - .unwrap(); - } - Command::Show(Show { path }) => { - let stats = DeltaTableBuilder::from_uri(path).load().await.unwrap(); - let ctx = SessionContext::new(); - ctx.register_table("stats", Arc::new(stats)).unwrap(); - - ctx.sql("select * from stats") - .await - .unwrap() - .show() - .await - .unwrap(); - } - } -} diff --git a/crates/deltalake-core/src/delta_datafusion/logical.rs b/crates/deltalake-core/src/delta_datafusion/logical.rs deleted file mode 100644 index 7b05dd57d9..0000000000 --- a/crates/deltalake-core/src/delta_datafusion/logical.rs +++ /dev/null @@ -1,48 +0,0 @@ -//! Logical Operations for DataFusion - -use datafusion_expr::{LogicalPlan, UserDefinedLogicalNodeCore}; - -// Metric Observer is used to update DataFusion metrics from a record batch. -// See MetricObserverExec for the physical implementation - -#[derive(Debug, Hash, Eq, PartialEq)] -pub(crate) struct MetricObserver { - // id is preserved during conversion to physical node - pub id: String, - pub input: LogicalPlan, -} - -impl UserDefinedLogicalNodeCore for MetricObserver { - // Predicate push down is not supported for this node. Try to limit usage - // near the end of plan. - fn name(&self) -> &str { - "MetricObserver" - } - - fn inputs(&self) -> Vec<&datafusion_expr::LogicalPlan> { - vec![&self.input] - } - - fn schema(&self) -> &datafusion_common::DFSchemaRef { - self.input.schema() - } - - fn expressions(&self) -> Vec { - vec![] - } - - fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "MetricObserver id={}", &self.id) - } - - fn from_template( - &self, - _exprs: &[datafusion_expr::Expr], - inputs: &[datafusion_expr::LogicalPlan], - ) -> Self { - MetricObserver { - id: self.id.clone(), - input: inputs[0].clone(), - } - } -} diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 8dea811383..1410efbfbc 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -81,8 +81,6 @@ use crate::{open_table, open_table_with_storage_options, DeltaTable}; const PATH_COLUMN: &str = "__delta_rs_path"; pub mod expr; -pub mod logical; -pub mod physical; impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { @@ -353,7 +351,7 @@ pub(crate) fn logical_schema( snapshot: &DeltaTableState, scan_config: &DeltaScanConfig, ) -> DeltaResult { - let input_schema = snapshot.arrow_schema()?; + let input_schema = snapshot.input_schema()?; let mut fields = Vec::new(); for field in input_schema.fields.iter() { fields.push(field.to_owned()); @@ -507,6 +505,11 @@ impl<'a> DeltaScanBuilder<'a> { self } + pub fn with_schema(mut self, schema: SchemaRef) -> Self { + self.schema = Some(schema); + self + } + pub async fn build(self) -> DeltaResult { let config = self.config; let schema = match self.schema { diff --git a/crates/deltalake-core/src/delta_datafusion/physical.rs b/crates/deltalake-core/src/delta_datafusion/physical.rs deleted file mode 100644 index 954df0b046..0000000000 --- a/crates/deltalake-core/src/delta_datafusion/physical.rs +++ /dev/null @@ -1,180 +0,0 @@ -//! Physical Operations for DataFusion -use std::sync::Arc; - -use arrow_schema::SchemaRef; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::error::Result as DataFusionResult; -use datafusion::physical_plan::DisplayAs; -use datafusion::physical_plan::{ - metrics::{ExecutionPlanMetricsSet, MetricsSet}, - ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, -}; -use futures::{Stream, StreamExt}; - -use crate::DeltaTableError; - -// Metric Observer is used to update DataFusion metrics from a record batch. -// Typically the null count for a particular column is pulled after performing a -// projection since this count is easy to obtain - -pub(crate) type MetricObserverFunction = fn(&RecordBatch, &ExecutionPlanMetricsSet) -> (); - -pub(crate) struct MetricObserverExec { - parent: Arc, - id: String, - metrics: ExecutionPlanMetricsSet, - update: MetricObserverFunction, -} - -impl MetricObserverExec { - pub fn new(id: String, parent: Arc, f: MetricObserverFunction) -> Self { - MetricObserverExec { - parent, - id, - metrics: ExecutionPlanMetricsSet::new(), - update: f, - } - } - - pub fn try_new( - id: String, - inputs: &[Arc], - f: MetricObserverFunction, - ) -> DataFusionResult> { - match inputs { - [input] => Ok(Arc::new(MetricObserverExec::new(id, input.clone(), f))), - _ => Err(datafusion_common::DataFusionError::External(Box::new( - DeltaTableError::Generic("MetricObserverExec expects only one child".into()), - ))), - } - } - - pub fn id(&self) -> &str { - &self.id - } -} - -impl std::fmt::Debug for MetricObserverExec { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("MetricObserverExec") - .field("id", &self.id) - .field("metrics", &self.metrics) - .finish() - } -} - -impl DisplayAs for MetricObserverExec { - fn fmt_as( - &self, - _: datafusion::physical_plan::DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "MetricObserverExec id={}", self.id) - } -} - -impl ExecutionPlan for MetricObserverExec { - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn schema(&self) -> arrow_schema::SchemaRef { - self.parent.schema() - } - - fn output_partitioning(&self) -> datafusion::physical_plan::Partitioning { - self.parent.output_partitioning() - } - - fn output_ordering(&self) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { - self.parent.output_ordering() - } - - fn children(&self) -> Vec> { - vec![self.parent.clone()] - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> datafusion_common::Result { - let res = self.parent.execute(partition, context)?; - Ok(Box::pin(MetricObserverStream { - schema: self.schema(), - input: res, - metrics: self.metrics.clone(), - update: self.update, - })) - } - - fn statistics(&self) -> DataFusionResult { - self.parent.statistics() - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> datafusion_common::Result> { - MetricObserverExec::try_new(self.id.clone(), &children, self.update) - } - - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } -} - -struct MetricObserverStream { - schema: SchemaRef, - input: SendableRecordBatchStream, - metrics: ExecutionPlanMetricsSet, - update: MetricObserverFunction, -} - -impl Stream for MetricObserverStream { - type Item = DataFusionResult; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - self.input.poll_next_unpin(cx).map(|x| match x { - Some(Ok(batch)) => { - (self.update)(&batch, &self.metrics); - Some(Ok(batch)) - } - other => other, - }) - } - - fn size_hint(&self) -> (usize, Option) { - self.input.size_hint() - } -} - -impl RecordBatchStream for MetricObserverStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -pub(crate) fn find_metric_node( - id: &str, - parent: &Arc, -) -> Option> { - //! Used to locate the physical MetricCountExec Node after the planner converts the logical node - if let Some(metric) = parent.as_any().downcast_ref::() { - if metric.id().eq(id) { - return Some(parent.to_owned()); - } - } - - for child in &parent.children() { - let res = find_metric_node(id, child); - if res.is_some() { - return res; - } - } - - None -} diff --git a/crates/deltalake-core/src/kernel/arrow/mod.rs b/crates/deltalake-core/src/kernel/actions/arrow.rs similarity index 98% rename from crates/deltalake-core/src/kernel/arrow/mod.rs rename to crates/deltalake-core/src/kernel/actions/arrow.rs index 0c89f6ab48..d292362604 100644 --- a/crates/deltalake-core/src/kernel/arrow/mod.rs +++ b/crates/deltalake-core/src/kernel/actions/arrow.rs @@ -1,5 +1,3 @@ -//! Conversions between Delta and Arrow data types - use std::sync::Arc; use arrow_schema::{ @@ -8,12 +6,7 @@ use arrow_schema::{ }; use lazy_static::lazy_static; -use super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; - -pub mod schemas; - -const MAP_KEYS_NAME: &str = "keys"; -const MAP_VALUES_NAME: &str = "values"; +use super::super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; impl TryFrom<&StructType> for ArrowSchema { type Error = ArrowError; @@ -71,9 +64,9 @@ impl TryFrom<&MapType> for ArrowField { "entries", ArrowDataType::Struct( vec![ - ArrowField::new(MAP_KEYS_NAME, ArrowDataType::try_from(a.key_type())?, false), + ArrowField::new("key", ArrowDataType::try_from(a.key_type())?, false), ArrowField::new( - MAP_VALUES_NAME, + "value", ArrowDataType::try_from(a.value_type())?, a.value_contains_null(), ), @@ -150,12 +143,12 @@ impl TryFrom<&DataType> for ArrowDataType { ArrowDataType::Struct( vec![ ArrowField::new( - MAP_KEYS_NAME, + "keys", >::try_from(m.key_type())?, false, ), ArrowField::new( - MAP_VALUES_NAME, + "values", >::try_from(m.value_type())?, m.value_contains_null(), ), diff --git a/crates/deltalake-core/src/kernel/actions/mod.rs b/crates/deltalake-core/src/kernel/actions/mod.rs index 637d520c41..865c9d3cd9 100644 --- a/crates/deltalake-core/src/kernel/actions/mod.rs +++ b/crates/deltalake-core/src/kernel/actions/mod.rs @@ -7,7 +7,9 @@ use std::collections::HashMap; use serde::{Deserialize, Serialize}; -pub(crate) mod schemas; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub(crate) mod arrow; +// pub(crate) mod schemas; mod serde_path; pub(crate) mod types; @@ -30,12 +32,10 @@ pub enum ActionType { Protocol, /// modify the data in a table by removing individual logical files Remove, + /// The Row ID high-water mark tracks the largest ID that has been assigned to a row in the table. + RowIdHighWaterMark, /// Transactional information Txn, - /// Checkpoint metadata - CheckpointMetadata, - /// Sidecar - Sidecar, } #[derive(Debug, PartialEq, Eq, Clone, Deserialize, Serialize)] diff --git a/crates/deltalake-core/src/kernel/actions/schemas.rs b/crates/deltalake-core/src/kernel/actions/schemas.rs index ad3e3ccbad..0cc870318f 100644 --- a/crates/deltalake-core/src/kernel/actions/schemas.rs +++ b/crates/deltalake-core/src/kernel/actions/schemas.rs @@ -1,262 +1,255 @@ -//! Schema definitions for action types +use std::sync::Arc; -use lazy_static::lazy_static; +use arrow_schema::{DataType, Field, Fields, Schema}; use super::ActionType; -use crate::kernel::schema::{ArrayType, DataType, MapType, StructField, StructType}; -lazy_static! { - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#change-metadata - static ref METADATA_FIELD: StructField = StructField::new( - "metaData", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("id", DataType::string(), false), - StructField::new("name", DataType::string(), true), - StructField::new("description", DataType::string(), true), - StructField::new( - "format", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("provider", DataType::string(), false), - StructField::new( - "configuration", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), - true, - ))), - true, +impl ActionType { + /// Returns the root field for the action type + pub fn field(&self) -> Field { + match self { + Self::Add => get_root("add", self.fields()), + Self::Cdc => get_root("cdc", self.fields()), + Self::CommitInfo => get_root("commitInfo", self.fields()), + Self::DomainMetadata => get_root("domainMetadata", self.fields()), + Self::Metadata => get_root("metaData", self.fields()), + Self::Protocol => get_root("protocol", self.fields()), + Self::Remove => get_root("remove", self.fields()), + Self::RowIdHighWaterMark => get_root("rowIdHighWaterMark", self.fields()), + Self::Txn => get_root("txn", self.fields()), + } + } + + /// Returns the child fields for the action type + pub fn fields(&self) -> Vec { + match self { + Self::Add => add_fields(), + Self::Cdc => cdc_fields(), + Self::CommitInfo => commit_info_fields(), + Self::DomainMetadata => domain_metadata_fields(), + Self::Metadata => metadata_fields(), + Self::Protocol => protocol_fields(), + Self::Remove => remove_fields(), + Self::RowIdHighWaterMark => watermark_fields(), + Self::Txn => txn_fields(), + } + } +} + +/// Returns the schema for the delta log +pub fn get_log_schema() -> Schema { + Schema { + fields: Fields::from_iter([ + ActionType::Add.field(), + ActionType::Cdc.field(), + ActionType::CommitInfo.field(), + ActionType::DomainMetadata.field(), + ActionType::Metadata.field(), + ActionType::Protocol.field(), + ActionType::Remove.field(), + ActionType::RowIdHighWaterMark.field(), + ActionType::Txn.field(), + ]), + metadata: Default::default(), + } +} + +fn get_root(name: &str, fields: Vec) -> Field { + Field::new(name, DataType::Struct(Fields::from_iter(fields)), true) +} + +fn add_fields() -> Vec { + Vec::from_iter([ + Field::new("path", DataType::Utf8, false), + Field::new("size", DataType::Int64, false), + Field::new("modificationTime", DataType::Int64, false), + Field::new("dataChange", DataType::Boolean, false), + Field::new("stats", DataType::Utf8, true), + Field::new( + "partitionValues", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + Field::new( + "tags", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + Field::new( + "deletionVector", + DataType::Struct(Fields::from(vec![ + Field::new("storageType", DataType::Utf8, false), + Field::new("pathOrInlineDv", DataType::Utf8, false), + Field::new("offset", DataType::Int32, true), + Field::new("sizeInBytes", DataType::Int32, false), + Field::new("cardinality", DataType::Int64, false), + ])), + true, + ), + Field::new("baseRowId", DataType::Int64, true), + Field::new("defaultRowCommitVersion", DataType::Int64, true), + ]) +} + +fn cdc_fields() -> Vec { + Vec::from_iter([ + Field::new("path", DataType::Utf8, true), + Field::new( + "partitionValues", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + Field::new("size", DataType::Int64, true), + Field::new("dataChange", DataType::Boolean, true), + Field::new( + "tags", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + ]) +} + +fn remove_fields() -> Vec { + Vec::from_iter([ + Field::new("path", DataType::Utf8, true), + Field::new("deletionTimestamp", DataType::Int64, true), + Field::new("dataChange", DataType::Boolean, true), + Field::new("extendedFileMetadata", DataType::Boolean, true), + Field::new("size", DataType::Int64, true), + Field::new( + "partitionValues", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + Field::new( + "tags", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + ]) +} + +fn metadata_fields() -> Vec { + Vec::from_iter([ + Field::new("id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, true), + Field::new("description", DataType::Utf8, true), + Field::new( + "format", + DataType::Struct(Fields::from_iter([ + Field::new("provider", DataType::Utf8, true), + Field::new( + "options", + DataType::Map( + Arc::new(Field::new( + "key_value", + DataType::Struct(Fields::from_iter([ + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Utf8, true), + ])), + false, + )), + false, ), - ]))), + false, + ), + ])), + false, + ), + Field::new("schemaString", DataType::Utf8, false), + Field::new("createdTime", DataType::Int64, true), + Field::new( + "partitionColumns", + DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), + false, + ), + Field::new( + "configuration", + DataType::Map( + Arc::new(Field::new( + "key_value", + DataType::Struct(Fields::from_iter([ + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Utf8, true), + ])), + false, + )), false, ), - StructField::new("schemaString", DataType::string(), false), - StructField::new( - "partitionColumns", - DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), - false, - ), - StructField::new("createdTime", DataType::long(), true), - StructField::new( - "configuration", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), - true, - ))), - false, - ), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#protocol-evolution - static ref PROTOCOL_FIELD: StructField = StructField::new( - "protocol", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("minReaderVersion", DataType::integer(), false), - StructField::new("minWriterVersion", DataType::integer(), false), - StructField::new( - "readerFeatures", - DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), - true, - ), - StructField::new( - "writerFeatures", - DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), - true, - ), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#commit-provenance-information - static ref COMMIT_INFO_FIELD: StructField = StructField::new( - "commitInfo", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("timestamp", DataType::timestamp(), false), - StructField::new("operation", DataType::string(), false), - StructField::new("isolationLevel", DataType::string(), true), - StructField::new("isBlindAppend", DataType::boolean(), true), - StructField::new("txnId", DataType::string(), true), - StructField::new("readVersion", DataType::long(), true), - StructField::new( - "operationParameters", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), - true, - ))), - true, - ), - StructField::new( - "operationMetrics", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), - true, - ))), - true, - ), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file - static ref ADD_FIELD: StructField = StructField::new( - "add", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("path", DataType::string(), false), - partition_values_field(), - StructField::new("size", DataType::long(), false), - StructField::new("modificationTime", DataType::timestamp(), false), - StructField::new("dataChange", DataType::boolean(), false), - StructField::new("stats", DataType::string(), true), - tags_field(), - deletion_vector_field(), - StructField::new("baseRowId", DataType::long(), true), - StructField::new("defaultRowCommitVersion", DataType::long(), true), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file - static ref REMOVE_FIELD: StructField = StructField::new( - "remove", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("path", DataType::string(), false), - StructField::new("deletionTimestamp", DataType::timestamp(), true), - StructField::new("dataChange", DataType::boolean(), false), - StructField::new("extendedFileMetadata", DataType::boolean(), true), - partition_values_field(), - StructField::new("size", DataType::long(), true), - StructField::new("stats", DataType::string(), true), - tags_field(), - deletion_vector_field(), - StructField::new("baseRowId", DataType::long(), true), - StructField::new("defaultRowCommitVersion", DataType::long(), true), - ]))), - true, - ); - static ref REMOVE_FIELD_CHECKPOINT: StructField = StructField::new( - "remove", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("path", DataType::string(), false), - StructField::new("deletionTimestamp", DataType::timestamp(), true), - StructField::new("dataChange", DataType::boolean(), false), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-cdc-file - static ref CDC_FIELD: StructField = StructField::new( - "cdc", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("path", DataType::string(), false), - partition_values_field(), - StructField::new("size", DataType::long(), false), - StructField::new("dataChange", DataType::boolean(), false), - tags_field(), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#transaction-identifiers - static ref TXN_FIELD: StructField = StructField::new( - "txn", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("appId", DataType::string(), false), - StructField::new("version", DataType::long(), false), - StructField::new("lastUpdated", DataType::timestamp(), true), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#domain-metadata - static ref DOMAIN_METADATA_FIELD: StructField = StructField::new( - "domainMetadata", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("domain", DataType::string(), false), - StructField::new( - "configuration", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), - true, - ))), - false, - ), - StructField::new("removed", DataType::boolean(), false), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#checkpoint-metadata - static ref CHECKPOINT_METADATA_FIELD: StructField = StructField::new( - "checkpointMetadata", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("flavor", DataType::string(), false), - tags_field(), - ]))), - true, - ); - // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#sidecar-file-information - static ref SIDECAR_FIELD: StructField = StructField::new( - "sidecar", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("path", DataType::string(), false), - StructField::new("sizeInBytes", DataType::long(), false), - StructField::new("modificationTime", DataType::timestamp(), false), - StructField::new("type", DataType::string(), false), - tags_field(), - ]))), - true, - ); + true, + ), + ]) } -fn tags_field() -> StructField { - StructField::new( - "tags", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), +fn protocol_fields() -> Vec { + Vec::from_iter([ + Field::new("minReaderVersion", DataType::Int32, false), + Field::new("minWriterVersion", DataType::Int32, false), + Field::new( + "readerFeatures", + DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), true, - ))), - true, - ) + ), + Field::new( + "writerFeatures", + DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), + true, + ), + ]) +} + +fn txn_fields() -> Vec { + Vec::from_iter([ + Field::new("appId", DataType::Utf8, true), + Field::new("version", DataType::Int64, true), + Field::new("lastUpdated", DataType::Int64, true), + ]) +} + +fn watermark_fields() -> Vec { + Vec::from_iter([Field::new("highWaterMark", DataType::Int64, true)]) } -fn partition_values_field() -> StructField { - StructField::new( - "partitionValues", - DataType::Map(Box::new(MapType::new( - DataType::string(), - DataType::string(), +fn commit_info_fields() -> Vec { + Vec::from_iter([ + Field::new("timestamp", DataType::Int64, true), + Field::new("operation", DataType::Utf8, true), + Field::new("isolationLevel", DataType::Utf8, true), + Field::new("isBlindAppend", DataType::Boolean, true), + Field::new("txnId", DataType::Utf8, true), + Field::new("readVersion", DataType::Int32, true), + Field::new( + "operationParameters", + DataType::Map(Arc::new(get_map_field()), false), true, - ))), - false, - ) + ), + Field::new( + "operationMetrics", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + ]) } -fn deletion_vector_field() -> StructField { - StructField::new( - "deletionVector", - DataType::Struct(Box::new(StructType::new(vec![ - StructField::new("storageType", DataType::string(), false), - StructField::new("pathOrInlineDv", DataType::string(), false), - StructField::new("offset", DataType::integer(), true), - StructField::new("sizeInBytes", DataType::integer(), false), - StructField::new("cardinality", DataType::long(), false), - ]))), - true, - ) +fn domain_metadata_fields() -> Vec { + Vec::from_iter([ + Field::new("domain", DataType::Utf8, true), + Field::new( + "configuration", + DataType::Map(Arc::new(get_map_field()), false), + true, + ), + Field::new("removed", DataType::Boolean, true), + ]) } -impl ActionType { - /// Returns the type of the corresponding field in the delta log schema - pub fn schema_field(&self) -> &StructField { - match self { - Self::Metadata => &METADATA_FIELD, - Self::Protocol => &PROTOCOL_FIELD, - Self::CommitInfo => &COMMIT_INFO_FIELD, - Self::Add => &ADD_FIELD, - Self::Remove => &REMOVE_FIELD, - Self::Cdc => &CDC_FIELD, - Self::Txn => &TXN_FIELD, - Self::DomainMetadata => &DOMAIN_METADATA_FIELD, - Self::CheckpointMetadata => &CHECKPOINT_METADATA_FIELD, - Self::Sidecar => &SIDECAR_FIELD, - } - } +fn get_map_field() -> Field { + Field::new( + "key_value", + DataType::Struct(Fields::from_iter([ + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Utf8, true), + ])), + false, + ) } diff --git a/crates/deltalake-core/src/kernel/actions/types.rs b/crates/deltalake-core/src/kernel/actions/types.rs index aa60823e4a..a788315b82 100644 --- a/crates/deltalake-core/src/kernel/actions/types.rs +++ b/crates/deltalake-core/src/kernel/actions/types.rs @@ -174,7 +174,7 @@ pub enum ReaderFeatures { /// Mapping of one column to another ColumnMapping, /// Deletion vectors for merge, update, delete - DeletionVectors, + DeleteionVecotrs, /// timestamps without timezone support #[serde(alias = "timestampNtz")] TimestampWithoutTimezone, @@ -185,13 +185,26 @@ pub enum ReaderFeatures { Other(String), } +#[allow(clippy::from_over_into)] +impl Into for ReaderFeatures { + fn into(self) -> usize { + match self { + ReaderFeatures::Other(_) => 0, + ReaderFeatures::ColumnMapping => 2, + ReaderFeatures::DeleteionVecotrs + | ReaderFeatures::TimestampWithoutTimezone + | ReaderFeatures::V2Checkpoint => 3, + } + } +} + #[cfg(all(not(feature = "parquet2"), feature = "parquet"))] impl From<&parquet::record::Field> for ReaderFeatures { fn from(value: &parquet::record::Field) -> Self { match value { parquet::record::Field::Str(feature) => match feature.as_str() { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeletionVectors, + "deletionVectors" => ReaderFeatures::DeleteionVecotrs, "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -203,15 +216,9 @@ impl From<&parquet::record::Field> for ReaderFeatures { impl From for ReaderFeatures { fn from(value: String) -> Self { - value.as_str().into() - } -} - -impl From<&str> for ReaderFeatures { - fn from(value: &str) -> Self { - match value { + match value.as_str() { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeletionVectors, + "deletionVectors" => ReaderFeatures::DeleteionVecotrs, "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -223,7 +230,7 @@ impl AsRef for ReaderFeatures { fn as_ref(&self) -> &str { match self { ReaderFeatures::ColumnMapping => "columnMapping", - ReaderFeatures::DeletionVectors => "deletionVectors", + ReaderFeatures::DeleteionVecotrs => "deletionVectors", ReaderFeatures::TimestampWithoutTimezone => "timestampNtz", ReaderFeatures::V2Checkpoint => "v2Checkpoint", ReaderFeatures::Other(f) => f, @@ -257,7 +264,7 @@ pub enum WriterFeatures { /// ID Columns IdentityColumns, /// Deletion vectors for merge, update, delete - DeletionVectors, + DeleteionVecotrs, /// Row tracking on tables RowTracking, /// timestamps without timezone support @@ -274,15 +281,29 @@ pub enum WriterFeatures { Other(String), } -impl From for WriterFeatures { - fn from(value: String) -> Self { - value.as_str().into() +#[allow(clippy::from_over_into)] +impl Into for WriterFeatures { + fn into(self) -> usize { + match self { + WriterFeatures::Other(_) => 0, + WriterFeatures::AppendOnly | WriterFeatures::Invariants => 2, + WriterFeatures::CheckConstraints => 3, + WriterFeatures::ChangeDataFeed | WriterFeatures::GeneratedColumns => 4, + WriterFeatures::ColumnMapping => 5, + WriterFeatures::IdentityColumns + | WriterFeatures::DeleteionVecotrs + | WriterFeatures::RowTracking + | WriterFeatures::TimestampWithoutTimezone + | WriterFeatures::DomainMetadata + | WriterFeatures::V2Checkpoint + | WriterFeatures::IcebergCompatV1 => 7, + } } } -impl From<&str> for WriterFeatures { - fn from(value: &str) -> Self { - match value { +impl From for WriterFeatures { + fn from(value: String) -> Self { + match value.as_str() { "appendOnly" => WriterFeatures::AppendOnly, "invariants" => WriterFeatures::Invariants, "checkConstraints" => WriterFeatures::CheckConstraints, @@ -290,7 +311,7 @@ impl From<&str> for WriterFeatures { "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeletionVectors, + "deletionVectors" => WriterFeatures::DeleteionVecotrs, "rowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, @@ -311,7 +332,7 @@ impl AsRef for WriterFeatures { WriterFeatures::GeneratedColumns => "generatedColumns", WriterFeatures::ColumnMapping => "columnMapping", WriterFeatures::IdentityColumns => "identityColumns", - WriterFeatures::DeletionVectors => "deletionVectors", + WriterFeatures::DeleteionVecotrs => "deletionVectors", WriterFeatures::RowTracking => "rowTracking", WriterFeatures::TimestampWithoutTimezone => "timestampNtz", WriterFeatures::DomainMetadata => "domainMetadata", @@ -340,7 +361,7 @@ impl From<&parquet::record::Field> for WriterFeatures { "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeletionVectors, + "deletionVectors" => WriterFeatures::DeleteionVecotrs, "rowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, @@ -400,7 +421,7 @@ impl AsRef for StorageType { impl ToString for StorageType { fn to_string(&self) -> String { - self.as_ref().into() + self.as_ref().to_string() } } @@ -429,7 +450,6 @@ pub struct DeletionVectorDescriptor { /// Start of the data for this DV in number of bytes from the beginning of the file it is stored in. /// Always None (absent in JSON) when `storageType = 'i'`. - #[serde(skip_serializing_if = "Option::is_none")] pub offset: Option, /// Size of the serialized DV in bytes (raw data size, i.e. before base85 encoding, if inline). @@ -642,11 +662,9 @@ pub struct Remove { pub data_change: bool, /// The time this logical file was created, as milliseconds since the epoch. - #[serde(skip_serializing_if = "Option::is_none")] pub deletion_timestamp: Option, /// When true the fields `partition_values`, `size`, and `tags` are present - #[serde(skip_serializing_if = "Option::is_none")] pub extended_file_metadata: Option, /// A map from partition column to value for this logical file. @@ -668,11 +686,9 @@ pub struct Remove { /// Default generated Row ID of the first row in the file. The default generated Row IDs /// of the other rows in the file can be reconstructed by adding the physical index of the /// row within the file to the base Row ID - #[serde(skip_serializing_if = "Option::is_none")] pub base_row_id: Option, /// First commit version in which an add action with the same path was committed to the table. - #[serde(skip_serializing_if = "Option::is_none")] pub default_row_commit_version: Option, } @@ -691,18 +707,13 @@ pub struct AddCDCFile { /// absolute path to a CDC file #[serde(with = "serde_path")] pub path: String, - /// The size of this file in bytes pub size: i64, - /// A map from partition column to value for this file pub partition_values: HashMap>, - /// Should always be set to false because they do not change the underlying data of the table pub data_change: bool, - /// Map containing metadata about this file - #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, } @@ -713,12 +724,9 @@ pub struct AddCDCFile { pub struct Txn { /// A unique identifier for the application performing the transaction. pub app_id: String, - /// An application-specific numeric identifier for this transaction. pub version: i64, - /// The time when this transaction action was created in milliseconds since the Unix epoch. - #[serde(skip_serializing_if = "Option::is_none")] pub last_updated: Option, } @@ -731,39 +739,30 @@ pub struct CommitInfo { /// Timestamp in millis when the commit was created #[serde(skip_serializing_if = "Option::is_none")] pub timestamp: Option, - /// Id of the user invoking the commit #[serde(skip_serializing_if = "Option::is_none")] pub user_id: Option, - /// Name of the user invoking the commit #[serde(skip_serializing_if = "Option::is_none")] pub user_name: Option, - /// The operation performed during the #[serde(skip_serializing_if = "Option::is_none")] pub operation: Option, - /// Parameters used for table operation #[serde(skip_serializing_if = "Option::is_none")] pub operation_parameters: Option>, - /// Version of the table when the operation was started #[serde(skip_serializing_if = "Option::is_none")] pub read_version: Option, - /// The isolation level of the commit #[serde(skip_serializing_if = "Option::is_none")] pub isolation_level: Option, - /// TODO #[serde(skip_serializing_if = "Option::is_none")] pub is_blind_append: Option, - /// Delta engine which created the commit. #[serde(skip_serializing_if = "Option::is_none")] pub engine_info: Option, - /// Additional provenance information for the commit #[serde(flatten, default)] pub info: HashMap, @@ -775,50 +774,12 @@ pub struct CommitInfo { pub struct DomainMetadata { /// Identifier for this domain (system or user-provided) pub domain: String, - /// String containing configuration for the metadata domain pub configuration: String, - /// When `true` the action serves as a tombstone pub removed: bool, } -#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] -/// This action is only allowed in checkpoints following V2 spec. It describes the details about the checkpoint. -pub struct CheckpointMetadata { - /// The flavor of the V2 checkpoint. Allowed values: "flat". - pub flavor: String, - - /// Map containing any additional metadata about the v2 spec checkpoint. - #[serde(skip_serializing_if = "Option::is_none")] - pub tags: Option>>, -} - -/// The sidecar action references a sidecar file which provides some of the checkpoint's file actions. -/// This action is only allowed in checkpoints following V2 spec. -#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] -pub struct Sidecar { - /// The name of the sidecar file (not a path). - /// The file must reside in the _delta_log/_sidecars directory. - pub file_name: String, - - /// The size of the sidecar file in bytes - pub size_in_bytes: i64, - - /// The time this sidecar file was created, as milliseconds since the epoch. - pub modification_time: i64, - - /// Type of sidecar. Valid values are: "fileaction". - /// This could be extended in future to allow different kinds of sidecars. - #[serde(rename = "type")] - pub sidecar_type: String, - - /// Map containing any additional metadata about the checkpoint sidecar file. - #[serde(skip_serializing_if = "Option::is_none")] - pub tags: Option>>, -} - #[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] /// The isolation level applied during transaction pub enum IsolationLevel { diff --git a/crates/deltalake-core/src/kernel/arrow/schemas.rs b/crates/deltalake-core/src/kernel/arrow/schemas.rs deleted file mode 100644 index 80a29e065e..0000000000 --- a/crates/deltalake-core/src/kernel/arrow/schemas.rs +++ /dev/null @@ -1,63 +0,0 @@ -//! Arrow schemas for the delta log - -use arrow_schema::{Field, Fields, Schema}; -use lazy_static::lazy_static; - -use super::super::ActionType; - -lazy_static! { - static ref ARROW_METADATA_FIELD: Field = - ActionType::Metadata.schema_field().try_into().unwrap(); - static ref ARROW_PROTOCOL_FIELD: Field = - ActionType::Protocol.schema_field().try_into().unwrap(); - static ref ARROW_COMMIT_INFO_FIELD: Field = - ActionType::CommitInfo.schema_field().try_into().unwrap(); - static ref ARROW_ADD_FIELD: Field = ActionType::Add.schema_field().try_into().unwrap(); - static ref ARROW_REMOVE_FIELD: Field = ActionType::Remove.schema_field().try_into().unwrap(); - static ref ARROW_CDC_FIELD: Field = ActionType::Cdc.schema_field().try_into().unwrap(); - static ref ARROW_TXN_FIELD: Field = ActionType::Txn.schema_field().try_into().unwrap(); - static ref ARROW_DOMAIN_METADATA_FIELD: Field = ActionType::DomainMetadata - .schema_field() - .try_into() - .unwrap(); - static ref ARROW_CHECKPOINT_METADATA_FIELD: Field = ActionType::CheckpointMetadata - .schema_field() - .try_into() - .unwrap(); - static ref ARROW_SIDECAR_FIELD: Field = ActionType::Sidecar.schema_field().try_into().unwrap(); -} - -impl ActionType { - /// Returns the root field for the action type - pub fn arrow_field(&self) -> &Field { - match self { - Self::Metadata => &ARROW_METADATA_FIELD, - Self::Protocol => &ARROW_PROTOCOL_FIELD, - Self::CommitInfo => &ARROW_COMMIT_INFO_FIELD, - Self::Add => &ARROW_ADD_FIELD, - Self::Remove => &ARROW_REMOVE_FIELD, - Self::Cdc => &ARROW_CDC_FIELD, - Self::Txn => &ARROW_TXN_FIELD, - Self::DomainMetadata => &ARROW_DOMAIN_METADATA_FIELD, - Self::CheckpointMetadata => &ARROW_CHECKPOINT_METADATA_FIELD, - Self::Sidecar => &ARROW_SIDECAR_FIELD, - } - } -} - -/// Returns the schema for the delta log -pub fn get_log_schema() -> Schema { - Schema { - fields: Fields::from_iter([ - ActionType::Add.arrow_field().clone(), - ActionType::Cdc.arrow_field().clone(), - ActionType::CommitInfo.arrow_field().clone(), - ActionType::DomainMetadata.arrow_field().clone(), - ActionType::Metadata.arrow_field().clone(), - ActionType::Protocol.arrow_field().clone(), - ActionType::Remove.arrow_field().clone(), - ActionType::Txn.arrow_field().clone(), - ]), - metadata: Default::default(), - } -} diff --git a/crates/deltalake-core/src/kernel/error.rs b/crates/deltalake-core/src/kernel/error.rs index a37dbdae67..8ec799ca96 100644 --- a/crates/deltalake-core/src/kernel/error.rs +++ b/crates/deltalake-core/src/kernel/error.rs @@ -23,8 +23,9 @@ pub enum Error { #[error("Arrow error: {0}")] Parquet(#[from] parquet::errors::ParquetError), + #[cfg(feature = "object_store")] #[error("Error interacting with object store: {0}")] - ObjectStore(#[from] object_store::Error), + ObjectStore(object_store::Error), #[error("File not found: {0}")] FileNotFound(String), diff --git a/crates/deltalake-core/src/kernel/mod.rs b/crates/deltalake-core/src/kernel/mod.rs index 54f742c3fb..7785c273f9 100644 --- a/crates/deltalake-core/src/kernel/mod.rs +++ b/crates/deltalake-core/src/kernel/mod.rs @@ -1,8 +1,6 @@ //! Kernel module pub mod actions; -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub mod arrow; pub mod error; pub mod schema; diff --git a/crates/deltalake-core/src/operations/convert_to_delta.rs b/crates/deltalake-core/src/operations/convert_to_delta.rs index 644591727c..84fffa1578 100644 --- a/crates/deltalake-core/src/operations/convert_to_delta.rs +++ b/crates/deltalake-core/src/operations/convert_to_delta.rs @@ -27,7 +27,7 @@ use serde_json::{Map, Value}; use std::{ collections::{HashMap, HashSet}, num::TryFromIntError, - str::{FromStr, Utf8Error}, + str::Utf8Error, sync::Arc, }; @@ -82,20 +82,6 @@ pub enum PartitionStrategy { Hive, } -impl FromStr for PartitionStrategy { - type Err = DeltaTableError; - - fn from_str(s: &str) -> DeltaResult { - match s.to_ascii_lowercase().as_str() { - "hive" => Ok(PartitionStrategy::Hive), - _ => Err(DeltaTableError::Generic(format!( - "Invalid partition strategy provided {}", - s - ))), - } - } -} - /// Build an operation to convert a Parquet table to a [`DeltaTable`] in place pub struct ConvertToDeltaBuilder { log_store: Option, diff --git a/crates/deltalake-core/src/operations/merge.rs b/crates/deltalake-core/src/operations/merge.rs index 8b0dd56708..a9ad6a8655 100644 --- a/crates/deltalake-core/src/operations/merge.rs +++ b/crates/deltalake-core/src/operations/merge.rs @@ -8,7 +8,8 @@ //! specified matter. See [`MergeBuilder`] for more information //! //! *WARNING* The current implementation rewrites the entire delta table so only -//! use on small to medium sized tables. +//! use on small to medium sized tables. The solution also cannot take advantage +//! of multiple threads and is limited to a single single thread. //! Enhancements tracked at #850 //! //! # Example @@ -36,25 +37,27 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::{Instant, SystemTime, UNIX_EPOCH}; -use async_trait::async_trait; -use datafusion::datasource::provider_as_source; +use arrow_schema::SchemaRef; use datafusion::error::Result as DataFusionResult; -use datafusion::execution::context::{QueryPlanner, SessionConfig}; use datafusion::logical_expr::build_join_schema; -use datafusion::physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner}; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::{ execution::context::SessionState, physical_plan::{ + filter::FilterExec, + joins::{ + utils::{build_join_schema as physical_build_join_schema, JoinFilter}, + NestedLoopJoinExec, + }, metrics::{MetricBuilder, MetricsSet}, + projection::ProjectionExec, ExecutionPlan, }, prelude::{DataFrame, SessionContext}, }; -use datafusion_common::{Column, DFSchema, ScalarValue, TableReference}; +use datafusion_common::{Column, DFField, DFSchema, ScalarValue, TableReference}; use datafusion_expr::{col, conditional_expressions::CaseBuilder, lit, when, Expr, JoinType}; -use datafusion_expr::{ - Extension, LogicalPlan, LogicalPlanBuilder, UserDefinedLogicalNode, UNNAMED_TABLE, -}; +use datafusion_physical_expr::{create_physical_expr, expressions, PhysicalExpr}; use futures::future::BoxFuture; use parquet::file::properties::WriterProperties; use serde::Serialize; @@ -63,19 +66,15 @@ use serde_json::Value; use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; use super::transaction::{commit, PROTOCOL}; use crate::delta_datafusion::expr::{fmt_expr_to_sql, parse_predicate_expression}; -use crate::delta_datafusion::logical::MetricObserver; -use crate::delta_datafusion::physical::{find_metric_node, MetricObserverExec}; -use crate::delta_datafusion::{register_store, DeltaScanConfig, DeltaTableProvider}; +use crate::delta_datafusion::{register_store, DeltaScanBuilder}; use crate::kernel::{Action, Remove}; use crate::logstore::LogStoreRef; +use crate::operations::datafusion_utils::MetricObserverExec; use crate::operations::write::write_execution_plan; use crate::protocol::{DeltaOperation, MergePredicate}; use crate::table::state::DeltaTableState; use crate::{DeltaResult, DeltaTable, DeltaTableError}; -const SOURCE_COLUMN: &str = "__delta_rs_source"; -const TARGET_COLUMN: &str = "__delta_rs_target"; - const OPERATION_COLUMN: &str = "__delta_rs_operation"; const DELETE_COLUMN: &str = "__delta_rs_delete"; const TARGET_INSERT_COLUMN: &str = "__delta_rs_target_insert"; @@ -84,16 +83,11 @@ const TARGET_DELETE_COLUMN: &str = "__delta_rs_target_delete"; const TARGET_COPY_COLUMN: &str = "__delta_rs_target_copy"; const SOURCE_COUNT_METRIC: &str = "num_source_rows"; -const TARGET_COUNT_METRIC: &str = "num_target_rows"; const TARGET_COPY_METRIC: &str = "num_copied_rows"; const TARGET_INSERTED_METRIC: &str = "num_target_inserted_rows"; const TARGET_UPDATED_METRIC: &str = "num_target_updated_rows"; const TARGET_DELETED_METRIC: &str = "num_target_deleted_rows"; -const SOURCE_COUNT_ID: &str = "merge_source_count"; -const TARGET_COUNT_ID: &str = "merge_target_count"; -const OUTPUT_COUNT_ID: &str = "merge_output_count"; - /// Merge records into a Delta Table. pub struct MergeBuilder { /// The join predicate @@ -563,89 +557,6 @@ pub struct MergeMetrics { pub rewrite_time_ms: u64, } -struct MergeMetricExtensionPlanner {} - -#[async_trait] -impl ExtensionPlanner for MergeMetricExtensionPlanner { - async fn plan_extension( - &self, - _planner: &dyn PhysicalPlanner, - node: &dyn UserDefinedLogicalNode, - _logical_inputs: &[&LogicalPlan], - physical_inputs: &[Arc], - _session_state: &SessionState, - ) -> DataFusionResult>> { - if let Some(metric_observer) = node.as_any().downcast_ref::() { - if metric_observer.id.eq(SOURCE_COUNT_ID) { - return Ok(Some(MetricObserverExec::try_new( - SOURCE_COUNT_ID.into(), - physical_inputs, - |batch, metrics| { - MetricBuilder::new(metrics) - .global_counter(SOURCE_COUNT_METRIC) - .add(batch.num_rows()); - }, - )?)); - } - - if metric_observer.id.eq(TARGET_COUNT_ID) { - return Ok(Some(MetricObserverExec::try_new( - TARGET_COUNT_ID.into(), - physical_inputs, - |batch, metrics| { - MetricBuilder::new(metrics) - .global_counter(TARGET_COUNT_METRIC) - .add(batch.num_rows()); - }, - )?)); - } - - if metric_observer.id.eq(OUTPUT_COUNT_ID) { - return Ok(Some(MetricObserverExec::try_new( - OUTPUT_COUNT_ID.into(), - physical_inputs, - |batch, metrics| { - MetricBuilder::new(metrics) - .global_counter(TARGET_INSERTED_METRIC) - .add( - batch - .column_by_name(TARGET_INSERT_COLUMN) - .unwrap() - .null_count(), - ); - MetricBuilder::new(metrics) - .global_counter(TARGET_UPDATED_METRIC) - .add( - batch - .column_by_name(TARGET_UPDATE_COLUMN) - .unwrap() - .null_count(), - ); - MetricBuilder::new(metrics) - .global_counter(TARGET_DELETED_METRIC) - .add( - batch - .column_by_name(TARGET_DELETE_COLUMN) - .unwrap() - .null_count(), - ); - MetricBuilder::new(metrics) - .global_counter(TARGET_COPY_METRIC) - .add( - batch - .column_by_name(TARGET_COPY_COLUMN) - .unwrap() - .null_count(), - ); - }, - )?)); - } - } - - Ok(None) - } -} - #[allow(clippy::too_many_arguments)] async fn execute( predicate: Expression, @@ -678,61 +589,83 @@ async fn execute( // If the user specified any not_source_match operations then those // predicates also need to be considered when pruning - let source_name = match &source_alias { - Some(alias) => TableReference::bare(alias.to_string()), - None => TableReference::bare(UNNAMED_TABLE), - }; + let target = Arc::new( + DeltaScanBuilder::new(snapshot, log_store.clone(), &state) + .with_schema(snapshot.input_schema()?) + .build() + .await?, + ); + + let source = source.create_physical_plan().await?; + + let source_count = Arc::new(MetricObserverExec::new(source, |batch, metrics| { + MetricBuilder::new(metrics) + .global_counter(SOURCE_COUNT_METRIC) + .add(batch.num_rows()); + })); + + let mut expressions: Vec<(Arc, String)> = Vec::new(); + let source_schema = source_count.schema(); + + for (i, field) in source_schema.fields().into_iter().enumerate() { + expressions.push(( + Arc::new(expressions::Column::new(field.name(), i)), + field.name().clone(), + )); + } + expressions.push(( + Arc::new(expressions::Literal::new(true.into())), + "__delta_rs_source".to_owned(), + )); + let source = Arc::new(ProjectionExec::try_new(expressions, source_count.clone())?); - let target_name = match &target_alias { - Some(alias) => TableReference::bare(alias.to_string()), - None => TableReference::bare(UNNAMED_TABLE), + let mut expressions: Vec<(Arc, String)> = Vec::new(); + let target_schema = target.schema(); + for (i, field) in target_schema.fields().into_iter().enumerate() { + expressions.push(( + Arc::new(expressions::Column::new(field.name(), i)), + field.name().to_owned(), + )); + } + expressions.push(( + Arc::new(expressions::Literal::new(true.into())), + "__delta_rs_target".to_owned(), + )); + let target = Arc::new(ProjectionExec::try_new(expressions, target.clone())?); + + // TODO: Currently a NestedLoopJoin is used but we should target to support SortMergeJoin + // This would require rewriting the join predicate to only contain equality between left and right columns and pushing some filters down + // Ideally it would be nice if the optimizer / planner can pick the best join so maybe explore rewriting the entire operation using logical plans. + + // NLJ requires both sides to have one partition for outer joins + let target = Arc::new(CoalescePartitionsExec::new(target)); + let source = Arc::new(CoalescePartitionsExec::new(source)); + + let source_schema = match &source_alias { + Some(alias) => { + DFSchema::try_from_qualified_schema(TableReference::bare(alias), &source.schema())? + } + None => DFSchema::try_from(source.schema().as_ref().to_owned())?, }; - // This is only done to provide the source columns with a correct table reference. Just renaming the columns does not work - let source = - LogicalPlanBuilder::scan(source_name, provider_as_source(source.into_view()), None)? - .build()?; - - let source = LogicalPlan::Extension(Extension { - node: Arc::new(MetricObserver { - id: SOURCE_COUNT_ID.into(), - input: source, - }), - }); - - let source = DataFrame::new(state.clone(), source); - let source = source.with_column(SOURCE_COLUMN, lit(true))?; - - let target_provider = Arc::new(DeltaTableProvider::try_new( - snapshot.clone(), - log_store.clone(), - DeltaScanConfig::default(), - )?); - let target_provider = provider_as_source(target_provider); + let target_schema = match &target_alias { + Some(alias) => { + DFSchema::try_from_qualified_schema(TableReference::bare(alias), &target.schema())? + } + None => DFSchema::try_from(target.schema().as_ref().to_owned())?, + }; - let target = LogicalPlanBuilder::scan(target_name, target_provider, None)?.build()?; + let join_schema_df = build_join_schema(&source_schema, &target_schema, &JoinType::Full)?; - // TODO: This is here to prevent predicate pushdowns. In the future we can replace this node to allow pushdowns depending on which operations are being used. - let target = LogicalPlan::Extension(Extension { - node: Arc::new(MetricObserver { - id: TARGET_COUNT_ID.into(), - input: target, - }), - }); - let target = DataFrame::new(state.clone(), target); - let target = target.with_column(TARGET_COLUMN, lit(true))?; + let join_schema = + physical_build_join_schema(&source.schema(), &target.schema(), &JoinType::Full); + let (join_schema, join_order) = (join_schema.0, join_schema.1); - let source_schema = source.schema(); - let target_schema = target.schema(); - let join_schema_df = build_join_schema(source_schema, target_schema, &JoinType::Full)?; let predicate = match predicate { Expression::DataFusion(expr) => expr, Expression::String(s) => parse_predicate_expression(&join_schema_df, s, &state)?, }; - let join = source.join(target, JoinType::Full, &[], &[], Some(predicate.clone()))?; - let join_schema_df = join.schema().to_owned(); - let match_operations: Vec = match_operations .into_iter() .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) @@ -748,15 +681,40 @@ async fn execute( .map(|op| MergeOperation::try_from(op, &join_schema_df, &state, &target_alias)) .collect::, DeltaTableError>>()?; - let matched = col(SOURCE_COLUMN) + let predicate_expr = create_physical_expr( + &predicate, + &join_schema_df, + &join_schema, + state.execution_props(), + )?; + + let join_filter = JoinFilter::new(predicate_expr, join_order, join_schema); + let join: Arc = Arc::new(NestedLoopJoinExec::try_new( + source.clone(), + target.clone(), + Some(join_filter), + &datafusion_expr::JoinType::Full, + )?); + + // Project to include __delta_rs_operation which indicates which particular operation to perform on the column. + let mut expressions: Vec<(Arc, String)> = Vec::new(); + let schema = join.schema(); + for (i, field) in schema.fields().into_iter().enumerate() { + expressions.push(( + Arc::new(expressions::Column::new(field.name(), i)), + field.name().to_owned(), + )); + } + + let matched = col("__delta_rs_source") .is_true() - .and(col(TARGET_COLUMN).is_true()); - let not_matched_target = col(SOURCE_COLUMN) + .and(col("__delta_rs_target").is_true()); + let not_matched_target = col("__delta_rs_source") .is_true() - .and(col(TARGET_COLUMN).is_null()); - let not_matched_source = col(SOURCE_COLUMN) + .and(col("__delta_rs_target").is_null()); + let not_matched_source = col("__delta_rs_source") .is_null() - .and(col(TARGET_COLUMN)) + .and(col("__delta_rs_target")) .is_true(); // Plus 3 for the default operations for each match category @@ -853,10 +811,35 @@ async fn execute( let case = CaseBuilder::new(None, when_expr, then_expr, None).end()?; - let projection = join.with_column(OPERATION_COLUMN, case)?; + let case = create_physical_expr( + &case, + &join_schema_df, + &join.schema(), + state.execution_props(), + )?; + expressions.push((case, OPERATION_COLUMN.to_owned())); + let projection = Arc::new(ProjectionExec::try_new(expressions, join.clone())?); + + let mut f = join_schema_df.fields().to_owned(); + f.push(DFField::new_unqualified( + OPERATION_COLUMN, + arrow_schema::DataType::Int64, + false, + )); + let project_schema_df = DFSchema::new_with_metadata(f, HashMap::new())?; + + // Project again and include the original table schema plus a column to mark if row needs to be filtered before write + let mut expressions: Vec<(Arc, String)> = Vec::new(); + let schema = projection.schema(); + for (i, field) in schema.fields().into_iter().enumerate() { + expressions.push(( + Arc::new(expressions::Column::new(field.name(), i)), + field.name().to_owned(), + )); + } - let mut new_columns = projection; - let mut write_projection = Vec::new(); + let mut projection_map = HashMap::new(); + let mut f = project_schema_df.fields().clone(); for delta_field in snapshot.schema().unwrap().fields() { let mut when_expr = Vec::with_capacity(operations_size); @@ -870,6 +853,7 @@ async fn execute( }; let name = delta_field.name(); let column = Column::new(qualifier.clone(), name); + let field = project_schema_df.field_with_name(qualifier.as_ref(), name)?; for (idx, (operations, _)) in ops.iter().enumerate() { let op = operations @@ -889,9 +873,22 @@ async fn execute( ) .end()?; + let case = create_physical_expr( + &case, + &project_schema_df, + &projection.schema(), + state.execution_props(), + )?; + + projection_map.insert(delta_field.name(), expressions.len()); let name = "__delta_rs_c_".to_owned() + delta_field.name(); - write_projection.push(col(name.clone()).alias(delta_field.name())); - new_columns = new_columns.with_column(&name, case)?; + + f.push(DFField::new_unqualified( + &name, + field.data_type().clone(), + true, + )); + expressions.push((case, name)); } let mut insert_when = Vec::with_capacity(ops.len()); @@ -957,47 +954,168 @@ async fn execute( ); } - fn build_case(when: Vec, then: Vec) -> DataFusionResult { - CaseBuilder::new( + fn build_case( + when: Vec, + then: Vec, + schema: SchemaRef, + input_dfschema: &DFSchema, + state: &SessionState, + ) -> DataFusionResult> { + let case = CaseBuilder::new( Some(Box::new(col(OPERATION_COLUMN))), when, then, Some(Box::new(lit(false))), ) - .end() + .end()?; + + create_physical_expr(&case, input_dfschema, &schema, state.execution_props()) } - new_columns = new_columns.with_column(DELETE_COLUMN, build_case(delete_when, delete_then)?)?; - new_columns = - new_columns.with_column(TARGET_INSERT_COLUMN, build_case(insert_when, insert_then)?)?; - new_columns = - new_columns.with_column(TARGET_UPDATE_COLUMN, build_case(update_when, update_then)?)?; - new_columns = new_columns.with_column( + let schema = projection.schema(); + let input_dfschema = project_schema_df; + expressions.push(( + build_case( + delete_when, + delete_then, + schema.clone(), + &input_dfschema, + &state, + )?, + DELETE_COLUMN.to_owned(), + )); + f.push(DFField::new_unqualified( + DELETE_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + + expressions.push(( + build_case( + insert_when, + insert_then, + schema.clone(), + &input_dfschema, + &state, + )?, + TARGET_INSERT_COLUMN.to_owned(), + )); + f.push(DFField::new_unqualified( + TARGET_INSERT_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + + expressions.push(( + build_case( + update_when, + update_then, + schema.clone(), + &input_dfschema, + &state, + )?, + TARGET_UPDATE_COLUMN.to_owned(), + )); + f.push(DFField::new_unqualified( + TARGET_UPDATE_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + + expressions.push(( + build_case( + target_delete_when, + target_delete_then, + schema.clone(), + &input_dfschema, + &state, + )?, + TARGET_DELETE_COLUMN.to_owned(), + )); + f.push(DFField::new_unqualified( TARGET_DELETE_COLUMN, - build_case(target_delete_when, target_delete_then)?, + arrow_schema::DataType::Boolean, + true, + )); + + expressions.push(( + build_case( + copy_when, + copy_then, + schema.clone(), + &input_dfschema, + &state, + )?, + TARGET_COPY_COLUMN.to_owned(), + )); + f.push(DFField::new_unqualified( + TARGET_COPY_COLUMN, + arrow_schema::DataType::Boolean, + true, + )); + + let projection = Arc::new(ProjectionExec::try_new(expressions, projection.clone())?); + + let target_count_plan = Arc::new(MetricObserverExec::new(projection, |batch, metrics| { + MetricBuilder::new(metrics) + .global_counter(TARGET_INSERTED_METRIC) + .add( + batch + .column_by_name(TARGET_INSERT_COLUMN) + .unwrap() + .null_count(), + ); + MetricBuilder::new(metrics) + .global_counter(TARGET_UPDATED_METRIC) + .add( + batch + .column_by_name(TARGET_UPDATE_COLUMN) + .unwrap() + .null_count(), + ); + MetricBuilder::new(metrics) + .global_counter(TARGET_DELETED_METRIC) + .add( + batch + .column_by_name(TARGET_DELETE_COLUMN) + .unwrap() + .null_count(), + ); + MetricBuilder::new(metrics) + .global_counter(TARGET_COPY_METRIC) + .add( + batch + .column_by_name(TARGET_COPY_COLUMN) + .unwrap() + .null_count(), + ); + })); + + let write_schema_df = DFSchema::new_with_metadata(f, HashMap::new())?; + + let write_predicate = create_physical_expr( + &(col(DELETE_COLUMN).is_false()), + &write_schema_df, + &target_count_plan.schema(), + state.execution_props(), )?; - new_columns = new_columns.with_column(TARGET_COPY_COLUMN, build_case(copy_when, copy_then)?)?; - - let new_columns = new_columns.into_optimized_plan()?; - let operation_count = LogicalPlan::Extension(Extension { - node: Arc::new(MetricObserver { - id: OUTPUT_COUNT_ID.into(), - input: new_columns, - }), - }); - - let operation_count = DataFrame::new(state.clone(), operation_count); - let filtered = operation_count.filter(col(DELETE_COLUMN).is_false())?; - - let project = filtered.select(write_projection)?; - let optimized = &project.into_optimized_plan()?; - - let state = state.with_query_planner(Arc::new(MergePlanner {})); - let write = state.create_physical_plan(optimized).await?; + let filter: Arc = Arc::new(FilterExec::try_new( + write_predicate, + target_count_plan.clone(), + )?); - let err = || DeltaTableError::Generic("Unable to locate expected metric node".into()); - let source_count = find_metric_node(SOURCE_COUNT_ID, &write).ok_or_else(err)?; - let op_count = find_metric_node(OUTPUT_COUNT_ID, &write).ok_or_else(err)?; + let mut expressions: Vec<(Arc, String)> = Vec::new(); + for (key, value) in projection_map { + expressions.push(( + Arc::new(expressions::Column::new( + &("__delta_rs_c_".to_owned() + key), + value, + )), + key.to_owned(), + )); + } + // project filtered records to delta schema + let projection = Arc::new(ProjectionExec::try_new(expressions, filter.clone())?); // write projected records let table_partition_cols = current_metadata.partition_columns.clone(); @@ -1006,9 +1124,9 @@ async fn execute( let add_actions = write_execution_plan( snapshot, state.clone(), - write, + projection.clone(), table_partition_cols.clone(), - log_store.object_store(), + log_store.object_store().clone(), Some(snapshot.table_config().target_file_size() as usize), None, writer_properties, @@ -1045,7 +1163,7 @@ async fn execute( let mut version = snapshot.version(); let source_count_metrics = source_count.metrics().unwrap(); - let target_count_metrics = op_count.metrics().unwrap(); + let target_count_metrics = target_count_plan.metrics().unwrap(); fn get_metric(metrics: &MetricsSet, name: &str) -> usize { metrics.sum_by_name(name).map(|m| m.as_usize()).unwrap_or(0) } @@ -1082,25 +1200,6 @@ async fn execute( Ok(((actions, version), metrics)) } -// TODO: Abstract MergePlanner into DeltaPlanner to support other delta operations in the future. -struct MergePlanner {} - -#[async_trait] -impl QueryPlanner for MergePlanner { - async fn create_physical_plan( - &self, - logical_plan: &LogicalPlan, - session_state: &SessionState, - ) -> DataFusionResult> { - let planner = Arc::new(Box::new(DefaultPhysicalPlanner::with_extension_planners( - vec![Arc::new(MergeMetricExtensionPlanner {})], - ))); - planner - .create_physical_plan(logical_plan, session_state) - .await - } -} - impl std::future::IntoFuture for MergeBuilder { type Output = DeltaResult<(DeltaTable, MergeMetrics)>; type IntoFuture = BoxFuture<'static, Self::Output>; @@ -1112,9 +1211,7 @@ impl std::future::IntoFuture for MergeBuilder { PROTOCOL.can_write_to(&this.snapshot)?; let state = this.state.unwrap_or_else(|| { - //TODO: Datafusion's Hashjoin has some memory issues. Running with all cores results in a OoM. Can be removed when upstream improvemetns are made. - let config = SessionConfig::new().with_target_partitions(1); - let session = SessionContext::new_with_config(config); + let session = SessionContext::new(); // If a user provides their own their DF state then they must register the store themselves register_store(this.log_store.clone(), session.runtime_env()); @@ -1252,8 +1349,8 @@ mod tests { async fn assert_merge(table: DeltaTable, metrics: MergeMetrics) { assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 1); - assert!(metrics.num_target_files_added >= 1); + assert_eq!(table.get_file_uris().count(), 1); + assert_eq!(metrics.num_target_files_added, 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); assert_eq!(metrics.num_target_rows_updated, 3); @@ -1345,7 +1442,7 @@ mod tests { .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate("target.value = 1") + .predicate("target.value = arrow_cast(1, 'Int32')") .update("value", "target.value + cast(1 as int)") }) .unwrap() @@ -1373,7 +1470,9 @@ mod tests { ); assert_eq!( parameters["notMatchedBySourcePredicates"], - json!(r#"[{"actionType":"update","predicate":"target.value = 1"}]"#) + json!( + r#"[{"actionType":"update","predicate":"target.value = arrow_cast(1, 'Int32')"}]"# + ) ); assert_merge(table, metrics).await; @@ -1401,7 +1500,9 @@ mod tests { }) .unwrap() .when_not_matched_by_source_update(|update| { - update.predicate("value = 1").update("value", "value + 1") + update + .predicate("value = arrow_cast(1, 'Int32')") + .update("value", "value + cast(1 as int)") }) .unwrap() .when_not_matched_insert(|insert| { @@ -1442,8 +1543,8 @@ mod tests { .unwrap() .when_not_matched_by_source_update(|update| { update - .predicate("value = 1") - .update("value", "target.value + 1") + .predicate("value = arrow_cast(1, 'Int32')") + .update("value", "target.value + cast(1 as int)") }) .unwrap() .when_not_matched_insert(|insert| { @@ -1556,8 +1657,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 3); - assert!(metrics.num_target_files_added >= 3); + assert_eq!(table.get_file_uris().count(), 3); + assert_eq!(metrics.num_target_files_added, 3); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 1); assert_eq!(metrics.num_target_rows_updated, 3); @@ -1619,8 +1720,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 2); - assert!(metrics.num_target_files_added >= 2); + assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 2); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1683,8 +1784,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(table.get_file_uris().count() >= 2); - assert!(metrics.num_target_files_added >= 2); + assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 3); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1817,7 +1918,8 @@ mod tests { .unwrap(); assert_eq!(table.version(), 2); - assert!(metrics.num_target_files_added >= 2); + assert_eq!(table.get_file_uris().count(), 2); + assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); assert_eq!(metrics.num_target_rows_copied, 3); assert_eq!(metrics.num_target_rows_updated, 0); @@ -1847,77 +1949,4 @@ mod tests { let actual = get_data(&table).await; assert_batches_sorted_eq!(&expected, &actual); } - - #[tokio::test] - async fn test_merge_empty_table() { - let schema = get_arrow_schema(&None); - let table = setup_table(Some(vec!["modified"])).await; - - assert_eq!(table.version(), 0); - assert_eq!(table.get_file_uris().count(), 0); - - let ctx = SessionContext::new(); - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(arrow::array::StringArray::from(vec!["B", "C", "X"])), - Arc::new(arrow::array::Int32Array::from(vec![10, 20, 30])), - Arc::new(arrow::array::StringArray::from(vec![ - "2021-02-02", - "2023-07-04", - "2023-07-04", - ])), - ], - ) - .unwrap(); - let source = ctx.read_batch(batch).unwrap(); - - let (table, metrics) = DeltaOps(table) - .merge( - source, - col("target.id") - .eq(col("source.id")) - .and(col("target.modified").eq(lit("2021-02-02"))), - ) - .with_source_alias("source") - .with_target_alias("target") - .when_matched_update(|update| { - update - .update("value", col("source.value")) - .update("modified", col("source.modified")) - }) - .unwrap() - .when_not_matched_insert(|insert| { - insert - .set("id", col("source.id")) - .set("value", col("source.value")) - .set("modified", col("source.modified")) - }) - .unwrap() - .await - .unwrap(); - - assert_eq!(table.version(), 1); - assert!(table.get_file_uris().count() >= 2); - assert!(metrics.num_target_files_added >= 2); - assert_eq!(metrics.num_target_files_removed, 0); - assert_eq!(metrics.num_target_rows_copied, 0); - assert_eq!(metrics.num_target_rows_updated, 0); - assert_eq!(metrics.num_target_rows_inserted, 3); - assert_eq!(metrics.num_target_rows_deleted, 0); - assert_eq!(metrics.num_output_rows, 3); - assert_eq!(metrics.num_source_rows, 3); - - let expected = vec![ - "+----+-------+------------+", - "| id | value | modified |", - "+----+-------+------------+", - "| B | 10 | 2021-02-02 |", - "| C | 20 | 2023-07-04 |", - "| X | 30 | 2023-07-04 |", - "+----+-------+------------+", - ]; - let actual = get_data(&table).await; - assert_batches_sorted_eq!(&expected, &actual); - } } diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index 473ef1451b..fb0f25d379 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -209,9 +209,20 @@ impl AsRef for DeltaOps { #[cfg(feature = "datafusion")] mod datafusion_utils { + use std::sync::Arc; + + use arrow_schema::SchemaRef; + use datafusion::arrow::record_batch::RecordBatch; + use datafusion::error::Result as DataFusionResult; use datafusion::execution::context::SessionState; - use datafusion_common::DFSchema; + use datafusion::physical_plan::DisplayAs; + use datafusion::physical_plan::{ + metrics::{ExecutionPlanMetricsSet, MetricsSet}, + ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, + }; + use datafusion_common::{DFSchema, Statistics}; use datafusion_expr::Expr; + use futures::{Stream, StreamExt}; use crate::{delta_datafusion::expr::parse_predicate_expression, DeltaResult}; @@ -261,4 +272,127 @@ mod datafusion_utils { None => None, }) } + + pub(crate) type MetricObserverFunction = fn(&RecordBatch, &ExecutionPlanMetricsSet) -> (); + + pub(crate) struct MetricObserverExec { + parent: Arc, + metrics: ExecutionPlanMetricsSet, + update: MetricObserverFunction, + } + + impl MetricObserverExec { + pub fn new(parent: Arc, f: MetricObserverFunction) -> Self { + MetricObserverExec { + parent, + metrics: ExecutionPlanMetricsSet::new(), + update: f, + } + } + } + + impl std::fmt::Debug for MetricObserverExec { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("MergeStatsExec") + .field("parent", &self.parent) + .field("metrics", &self.metrics) + .finish() + } + } + + impl DisplayAs for MetricObserverExec { + fn fmt_as( + &self, + _: datafusion::physical_plan::DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + write!(f, "MetricObserverExec") + } + } + + impl ExecutionPlan for MetricObserverExec { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> arrow_schema::SchemaRef { + self.parent.schema() + } + + fn output_partitioning(&self) -> datafusion::physical_plan::Partitioning { + self.parent.output_partitioning() + } + + fn output_ordering(&self) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { + self.parent.output_ordering() + } + + fn children(&self) -> Vec> { + vec![self.parent.clone()] + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> datafusion_common::Result + { + let res = self.parent.execute(partition, context)?; + Ok(Box::pin(MetricObserverStream { + schema: self.schema(), + input: res, + metrics: self.metrics.clone(), + update: self.update, + })) + } + + fn statistics(&self) -> DataFusionResult { + self.parent.statistics() + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + ExecutionPlan::with_new_children(self.parent.clone(), children) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + } + + struct MetricObserverStream { + schema: SchemaRef, + input: SendableRecordBatchStream, + metrics: ExecutionPlanMetricsSet, + update: MetricObserverFunction, + } + + impl Stream for MetricObserverStream { + type Item = DataFusionResult; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + self.input.poll_next_unpin(cx).map(|x| match x { + Some(Ok(batch)) => { + (self.update)(&batch, &self.metrics); + Some(Ok(batch)) + } + other => other, + }) + } + + fn size_hint(&self) -> (usize, Option) { + self.input.size_hint() + } + } + + impl RecordBatchStream for MetricObserverStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + } } diff --git a/crates/deltalake-core/src/operations/update.rs b/crates/deltalake-core/src/operations/update.rs index 907dec5998..7583ed6b39 100644 --- a/crates/deltalake-core/src/operations/update.rs +++ b/crates/deltalake-core/src/operations/update.rs @@ -43,10 +43,10 @@ use parquet::file::properties::WriterProperties; use serde::Serialize; use serde_json::Value; -use super::datafusion_utils::Expression; +use super::datafusion_utils::{Expression, MetricObserverExec}; use super::transaction::{commit, PROTOCOL}; use super::write::write_execution_plan; -use crate::delta_datafusion::{expr::fmt_expr_to_sql, physical::MetricObserverExec}; +use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::kernel::{Action, Remove}; use crate::logstore::LogStoreRef; @@ -275,7 +275,6 @@ async fn execute( Arc::new(ProjectionExec::try_new(expressions, scan)?); let count_plan = Arc::new(MetricObserverExec::new( - "update_count".into(), projection_predicate.clone(), |batch, metrics| { let array = batch.column_by_name("__delta_rs_update_predicate").unwrap(); diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs index 837483c35c..a4cc1b66c7 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -18,7 +18,7 @@ use regex::Regex; use serde_json::Value; use super::{time_utils, ProtocolError}; -use crate::kernel::arrow::delta_log_schema_for_table; +use crate::kernel::actions::arrow::delta_log_schema_for_table; use crate::kernel::{ Action, Add as AddAction, DataType, Metadata, PrimitiveType, Protocol, StructField, StructType, Txn, diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index e2add9b529..8a5cd9f858 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -23,9 +23,8 @@ use std::borrow::Borrow; use std::collections::HashMap; use std::hash::{Hash, Hasher}; use std::mem::take; -use std::str::FromStr; -use crate::errors::{DeltaResult, DeltaTableError}; +use crate::errors::DeltaResult; use crate::kernel::{Add, CommitInfo, Metadata, Protocol, Remove}; use crate::logstore::LogStore; use crate::table::CheckPoint; @@ -590,23 +589,6 @@ pub enum SaveMode { Ignore, } -impl FromStr for SaveMode { - type Err = DeltaTableError; - - fn from_str(s: &str) -> DeltaResult { - match s.to_ascii_lowercase().as_str() { - "append" => Ok(SaveMode::Append), - "overwrite" => Ok(SaveMode::Overwrite), - "error" => Ok(SaveMode::ErrorIfExists), - "ignore" => Ok(SaveMode::Ignore), - _ => Err(DeltaTableError::Generic(format!( - "Invalid save mode provided: {}, only these are supported: ['append', 'overwrite', 'error', 'ignore']", - s - ))), - } - } -} - /// The OutputMode used in streaming operations. #[derive(Serialize, Deserialize, Debug, Clone)] pub enum OutputMode { diff --git a/crates/deltalake-core/src/schema/arrow_convert.rs b/crates/deltalake-core/src/schema/arrow_convert.rs new file mode 100644 index 0000000000..d292362604 --- /dev/null +++ b/crates/deltalake-core/src/schema/arrow_convert.rs @@ -0,0 +1,1049 @@ +use std::sync::Arc; + +use arrow_schema::{ + ArrowError, DataType as ArrowDataType, Field as ArrowField, FieldRef as ArrowFieldRef, + Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, TimeUnit, +}; +use lazy_static::lazy_static; + +use super::super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; + +impl TryFrom<&StructType> for ArrowSchema { + type Error = ArrowError; + + fn try_from(s: &StructType) -> Result { + let fields = s + .fields() + .iter() + .map(>::try_from) + .collect::, ArrowError>>()?; + + Ok(ArrowSchema::new(fields)) + } +} + +impl TryFrom<&StructField> for ArrowField { + type Error = ArrowError; + + fn try_from(f: &StructField) -> Result { + let metadata = f + .metadata() + .iter() + .map(|(key, val)| Ok((key.clone(), serde_json::to_string(val)?))) + .collect::>() + .map_err(|err| ArrowError::JsonError(err.to_string()))?; + + let field = ArrowField::new( + f.name(), + ArrowDataType::try_from(f.data_type())?, + f.is_nullable(), + ) + .with_metadata(metadata); + + Ok(field) + } +} + +impl TryFrom<&ArrayType> for ArrowField { + type Error = ArrowError; + + fn try_from(a: &ArrayType) -> Result { + Ok(ArrowField::new( + "item", + ArrowDataType::try_from(a.element_type())?, + a.contains_null(), + )) + } +} + +impl TryFrom<&MapType> for ArrowField { + type Error = ArrowError; + + fn try_from(a: &MapType) -> Result { + Ok(ArrowField::new( + "entries", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::try_from(a.key_type())?, false), + ArrowField::new( + "value", + ArrowDataType::try_from(a.value_type())?, + a.value_contains_null(), + ), + ] + .into(), + ), + false, // always non-null + )) + } +} + +impl TryFrom<&DataType> for ArrowDataType { + type Error = ArrowError; + + fn try_from(t: &DataType) -> Result { + match t { + DataType::Primitive(p) => { + match p { + PrimitiveType::String => Ok(ArrowDataType::Utf8), + PrimitiveType::Long => Ok(ArrowDataType::Int64), // undocumented type + PrimitiveType::Integer => Ok(ArrowDataType::Int32), + PrimitiveType::Short => Ok(ArrowDataType::Int16), + PrimitiveType::Byte => Ok(ArrowDataType::Int8), + PrimitiveType::Float => Ok(ArrowDataType::Float32), + PrimitiveType::Double => Ok(ArrowDataType::Float64), + PrimitiveType::Boolean => Ok(ArrowDataType::Boolean), + PrimitiveType::Binary => Ok(ArrowDataType::Binary), + PrimitiveType::Decimal(precision, scale) => { + let precision = u8::try_from(*precision).map_err(|_| { + ArrowError::SchemaError(format!( + "Invalid precision for decimal: {}", + precision + )) + })?; + let scale = i8::try_from(*scale).map_err(|_| { + ArrowError::SchemaError(format!("Invalid scale for decimal: {}", scale)) + })?; + + if precision <= 38 { + Ok(ArrowDataType::Decimal128(precision, scale)) + } else if precision <= 76 { + Ok(ArrowDataType::Decimal256(precision, scale)) + } else { + Err(ArrowError::SchemaError(format!( + "Precision too large to be represented in Arrow: {}", + precision + ))) + } + } + PrimitiveType::Date => { + // A calendar date, represented as a year-month-day triple without a + // timezone. Stored as 4 bytes integer representing days since 1970-01-01 + Ok(ArrowDataType::Date32) + } + PrimitiveType::Timestamp => { + // Issue: https://github.com/delta-io/delta/issues/643 + Ok(ArrowDataType::Timestamp(TimeUnit::Microsecond, None)) + } + } + } + DataType::Struct(s) => Ok(ArrowDataType::Struct( + s.fields() + .iter() + .map(>::try_from) + .collect::, ArrowError>>()? + .into(), + )), + DataType::Array(a) => Ok(ArrowDataType::List(Arc::new(>::try_from(a)?))), + DataType::Map(m) => Ok(ArrowDataType::Map( + Arc::new(ArrowField::new( + "entries", + ArrowDataType::Struct( + vec![ + ArrowField::new( + "keys", + >::try_from(m.key_type())?, + false, + ), + ArrowField::new( + "values", + >::try_from(m.value_type())?, + m.value_contains_null(), + ), + ] + .into(), + ), + false, + )), + false, + )), + } + } +} + +impl TryFrom<&ArrowSchema> for StructType { + type Error = ArrowError; + + fn try_from(arrow_schema: &ArrowSchema) -> Result { + let new_fields: Result, _> = arrow_schema + .fields() + .iter() + .map(|field| field.as_ref().try_into()) + .collect(); + Ok(StructType::new(new_fields?)) + } +} + +impl TryFrom for StructType { + type Error = ArrowError; + + fn try_from(arrow_schema: ArrowSchemaRef) -> Result { + arrow_schema.as_ref().try_into() + } +} + +impl TryFrom<&ArrowField> for StructField { + type Error = ArrowError; + + fn try_from(arrow_field: &ArrowField) -> Result { + Ok(StructField::new( + arrow_field.name().clone(), + arrow_field.data_type().try_into()?, + arrow_field.is_nullable(), + ) + .with_metadata(arrow_field.metadata().iter().map(|(k, v)| (k.clone(), v)))) + } +} + +impl TryFrom<&ArrowDataType> for DataType { + type Error = ArrowError; + + fn try_from(arrow_datatype: &ArrowDataType) -> Result { + match arrow_datatype { + ArrowDataType::Utf8 => Ok(DataType::Primitive(PrimitiveType::String)), + ArrowDataType::LargeUtf8 => Ok(DataType::Primitive(PrimitiveType::String)), + ArrowDataType::Int64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type + ArrowDataType::Int32 => Ok(DataType::Primitive(PrimitiveType::Integer)), + ArrowDataType::Int16 => Ok(DataType::Primitive(PrimitiveType::Short)), + ArrowDataType::Int8 => Ok(DataType::Primitive(PrimitiveType::Byte)), + ArrowDataType::UInt64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type + ArrowDataType::UInt32 => Ok(DataType::Primitive(PrimitiveType::Integer)), + ArrowDataType::UInt16 => Ok(DataType::Primitive(PrimitiveType::Short)), + ArrowDataType::UInt8 => Ok(DataType::Primitive(PrimitiveType::Boolean)), + ArrowDataType::Float32 => Ok(DataType::Primitive(PrimitiveType::Float)), + ArrowDataType::Float64 => Ok(DataType::Primitive(PrimitiveType::Double)), + ArrowDataType::Boolean => Ok(DataType::Primitive(PrimitiveType::Boolean)), + ArrowDataType::Binary => Ok(DataType::Primitive(PrimitiveType::Binary)), + ArrowDataType::FixedSizeBinary(_) => Ok(DataType::Primitive(PrimitiveType::Binary)), + ArrowDataType::LargeBinary => Ok(DataType::Primitive(PrimitiveType::Binary)), + ArrowDataType::Decimal128(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( + *p as i32, *s as i32, + ))), + ArrowDataType::Decimal256(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( + *p as i32, *s as i32, + ))), + ArrowDataType::Date32 => Ok(DataType::Primitive(PrimitiveType::Date)), + ArrowDataType::Date64 => Ok(DataType::Primitive(PrimitiveType::Date)), + ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => { + Ok(DataType::Primitive(PrimitiveType::Timestamp)) + } + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some(tz)) + if tz.eq_ignore_ascii_case("utc") => + { + Ok(DataType::Primitive(PrimitiveType::Timestamp)) + } + ArrowDataType::Struct(fields) => { + let converted_fields: Result, _> = fields + .iter() + .map(|field| field.as_ref().try_into()) + .collect(); + Ok(DataType::Struct(Box::new(StructType::new( + converted_fields?, + )))) + } + ArrowDataType::List(field) => Ok(DataType::Array(Box::new(ArrayType::new( + (*field).data_type().try_into()?, + (*field).is_nullable(), + )))), + ArrowDataType::LargeList(field) => Ok(DataType::Array(Box::new(ArrayType::new( + (*field).data_type().try_into()?, + (*field).is_nullable(), + )))), + ArrowDataType::FixedSizeList(field, _) => Ok(DataType::Array(Box::new( + ArrayType::new((*field).data_type().try_into()?, (*field).is_nullable()), + ))), + ArrowDataType::Map(field, _) => { + if let ArrowDataType::Struct(struct_fields) = field.data_type() { + let key_type = struct_fields[0].data_type().try_into()?; + let value_type = struct_fields[1].data_type().try_into()?; + let value_type_nullable = struct_fields[1].is_nullable(); + Ok(DataType::Map(Box::new(MapType::new( + key_type, + value_type, + value_type_nullable, + )))) + } else { + panic!("DataType::Map should contain a struct field child"); + } + } + s => Err(ArrowError::SchemaError(format!( + "Invalid data type for Delta Lake: {s}" + ))), + } + } +} + +macro_rules! arrow_map { + ($fieldname: ident, null) => { + ArrowField::new( + stringify!($fieldname), + ArrowDataType::Map( + Arc::new(ArrowField::new( + "entries", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::Utf8, false), + ArrowField::new("value", ArrowDataType::Utf8, true), + ] + .into(), + ), + false, + )), + false, + ), + true, + ) + }; + ($fieldname: ident, not_null) => { + ArrowField::new( + stringify!($fieldname), + ArrowDataType::Map( + Arc::new(ArrowField::new( + "entries", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::Utf8, false), + ArrowField::new("value", ArrowDataType::Utf8, false), + ] + .into(), + ), + false, + )), + false, + ), + false, + ) + }; +} + +macro_rules! arrow_field { + ($fieldname:ident, $type_qual:ident, null) => { + ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, true) + }; + ($fieldname:ident, $type_qual:ident, not_null) => { + ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, false) + }; +} + +macro_rules! arrow_list { + ($fieldname:ident, $element_name:ident, $type_qual:ident, null) => { + ArrowField::new( + stringify!($fieldname), + ArrowDataType::List(Arc::new(ArrowField::new( + stringify!($element_name), + ArrowDataType::$type_qual, + true, + ))), + true, + ) + }; + ($fieldname:ident, $element_name:ident, $type_qual:ident, not_null) => { + ArrowField::new( + stringify!($fieldname), + ArrowDataType::List(Arc::new(ArrowField::new( + stringify!($element_name), + ArrowDataType::$type_qual, + true, + ))), + false, + ) + }; +} + +macro_rules! arrow_struct { + ($fieldname:ident, [$($inner:tt)+], null) => { + ArrowField::new( + stringify!($fieldname), + ArrowDataType::Struct( + arrow_defs! [$($inner)+].into() + ), + true + ) + }; + ($fieldname:ident, [$($inner:tt)+], not_null) => { + ArrowField::new( + stringify!($fieldname), + ArrowDataType::Struct( + arrow_defs! [$($inner)+].into() + ), + false + ) + } +} + +macro_rules! arrow_def { + ($fieldname:ident $(null)?) => { + arrow_map!($fieldname, null) + }; + ($fieldname:ident not_null) => { + arrow_map!($fieldname, not_null) + }; + ($fieldname:ident[$inner_name:ident]{$type_qual:ident} $(null)?) => { + arrow_list!($fieldname, $inner_name, $type_qual, null) + }; + ($fieldname:ident[$inner_name:ident]{$type_qual:ident} not_null) => { + arrow_list!($fieldname, $inner_name, $type_qual, not_null) + }; + ($fieldname:ident:$type_qual:ident $(null)?) => { + arrow_field!($fieldname, $type_qual, null) + }; + ($fieldname:ident:$type_qual:ident not_null) => { + arrow_field!($fieldname, $type_qual, not_null) + }; + ($fieldname:ident[$($inner:tt)+] $(null)?) => { + arrow_struct!($fieldname, [$($inner)+], null) + }; + ($fieldname:ident[$($inner:tt)+] not_null) => { + arrow_struct!($fieldname, [$($inner)+], not_null) + } +} + +/// A helper macro to create more readable Arrow field definitions, delimited by commas +/// +/// The argument patterns are as follows: +/// +/// fieldname (null|not_null)? -- An arrow field of type map with name "fieldname" consisting of Utf8 key-value pairs, and an +/// optional nullability qualifier (null if not specified). +/// +/// fieldname:type (null|not_null)? -- An Arrow field consisting of an atomic type. For example, +/// id:Utf8 gets mapped to ArrowField::new("id", ArrowDataType::Utf8, true). +/// where customerCount:Int64 not_null gets mapped to gets mapped to +/// ArrowField::new("customerCount", ArrowDataType::Utf8, true) +/// +/// fieldname[list_element]{list_element_type} (null|not_null)? -- An Arrow list, with the name of the elements wrapped in square brackets +/// and the type of the list elements wrapped in curly brackets. For example, +/// customers[name]{Utf8} is an nullable arrow field of type arrow list consisting +/// of elements called "name" with type Utf8. +/// +/// fieldname[element1, element2, element3, ....] (null|not_null)? -- An arrow struct with name "fieldname" consisting of elements adhering to any of the patterns +/// documented, including additional structs arbitrarily nested up to the recursion +/// limit for Rust macros. +macro_rules! arrow_defs { + () => { + vec![] as Vec + }; + ($($fieldname:ident$(:$type_qual:ident)?$([$($inner:tt)+])?$({$list_type_qual:ident})? $($nullable:ident)?),+) => { + vec![ + $(arrow_def!($fieldname$(:$type_qual)?$([$($inner)+])?$({$list_type_qual})? $($nullable)?)),+ + ] + } +} + +/// Returns an arrow schema representing the delta log for use in checkpoints +/// +/// # Arguments +/// +/// * `table_schema` - The arrow schema representing the table backed by the delta log +/// * `partition_columns` - The list of partition columns of the table. +/// * `use_extended_remove_schema` - Whether to include extended file metadata in remove action schema. +/// Required for compatibility with different versions of Databricks runtime. +pub(crate) fn delta_log_schema_for_table( + table_schema: ArrowSchema, + partition_columns: &[String], + use_extended_remove_schema: bool, +) -> ArrowSchemaRef { + lazy_static! { + static ref SCHEMA_FIELDS: Vec = arrow_defs![ + metaData[ + id:Utf8, + name:Utf8, + description:Utf8, + schemaString:Utf8, + createdTime:Int64, + partitionColumns[element]{Utf8}, + configuration, + format[provider:Utf8, options] + ], + protocol[ + minReaderVersion:Int32, + minWriterVersion:Int32 + ], + txn[ + appId:Utf8, + version:Int64 + ] + ]; + static ref ADD_FIELDS: Vec = arrow_defs![ + path:Utf8, + size:Int64, + modificationTime:Int64, + dataChange:Boolean, + stats:Utf8, + partitionValues, + tags, + deletionVector[ + storageType:Utf8 not_null, + pathOrInlineDv:Utf8 not_null, + offset:Int32 null, + sizeInBytes:Int32 not_null, + cardinality:Int64 not_null + ] + ]; + static ref REMOVE_FIELDS: Vec = arrow_defs![ + path: Utf8, + deletionTimestamp: Int64, + dataChange: Boolean, + extendedFileMetadata: Boolean + ]; + static ref REMOVE_EXTENDED_FILE_METADATA_FIELDS: Vec = + arrow_defs![size: Int64, partitionValues, tags]; + }; + + // create add fields according to the specific data table schema + let (partition_fields, non_partition_fields): (Vec, Vec) = + table_schema + .fields() + .iter() + .map(|field| field.to_owned()) + .partition(|field| partition_columns.contains(field.name())); + + let mut stats_parsed_fields: Vec = + vec![ArrowField::new("numRecords", ArrowDataType::Int64, true)]; + if !non_partition_fields.is_empty() { + let mut max_min_vec = Vec::new(); + non_partition_fields + .iter() + .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); + + stats_parsed_fields.extend(["minValues", "maxValues"].into_iter().map(|name| { + ArrowField::new( + name, + ArrowDataType::Struct(max_min_vec.clone().into()), + true, + ) + })); + + let mut null_count_vec = Vec::new(); + non_partition_fields + .iter() + .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); + let null_count_struct = ArrowField::new( + "nullCount", + ArrowDataType::Struct(null_count_vec.into()), + true, + ); + + stats_parsed_fields.push(null_count_struct); + } + let mut add_fields = ADD_FIELDS.clone(); + add_fields.push(ArrowField::new( + "stats_parsed", + ArrowDataType::Struct(stats_parsed_fields.into()), + true, + )); + if !partition_fields.is_empty() { + add_fields.push(ArrowField::new( + "partitionValues_parsed", + ArrowDataType::Struct(partition_fields.into()), + true, + )); + } + + // create remove fields with or without extendedFileMetadata + let mut remove_fields = REMOVE_FIELDS.clone(); + if use_extended_remove_schema { + remove_fields.extend(REMOVE_EXTENDED_FILE_METADATA_FIELDS.clone()); + } + + // include add and remove fields in checkpoint schema + let mut schema_fields = SCHEMA_FIELDS.clone(); + schema_fields.push(ArrowField::new( + "add", + ArrowDataType::Struct(add_fields.into()), + true, + )); + schema_fields.push(ArrowField::new( + "remove", + ArrowDataType::Struct(remove_fields.into()), + true, + )); + + let arrow_schema = ArrowSchema::new(schema_fields); + + std::sync::Arc::new(arrow_schema) +} + +fn max_min_schema_for_fields(dest: &mut Vec, f: &ArrowField) { + match f.data_type() { + ArrowDataType::Struct(struct_fields) => { + let mut child_dest = Vec::new(); + + for f in struct_fields { + max_min_schema_for_fields(&mut child_dest, f); + } + + dest.push(ArrowField::new( + f.name(), + ArrowDataType::Struct(child_dest.into()), + true, + )); + } + // don't compute min or max for list, map or binary types + ArrowDataType::List(_) | ArrowDataType::Map(_, _) | ArrowDataType::Binary => { /* noop */ } + _ => { + let f = f.clone(); + dest.push(f); + } + } +} + +fn null_count_schema_for_fields(dest: &mut Vec, f: &ArrowField) { + match f.data_type() { + ArrowDataType::Struct(struct_fields) => { + let mut child_dest = Vec::new(); + + for f in struct_fields { + null_count_schema_for_fields(&mut child_dest, f); + } + + dest.push(ArrowField::new( + f.name(), + ArrowDataType::Struct(child_dest.into()), + true, + )); + } + _ => { + let f = ArrowField::new(f.name(), ArrowDataType::Int64, true); + dest.push(f); + } + } +} + +#[cfg(test)] +mod tests { + use arrow::array::ArrayData; + use arrow_array::Array; + use arrow_array::{make_array, ArrayRef, MapArray, StringArray, StructArray}; + use arrow_buffer::{Buffer, ToByteSlice}; + use arrow_schema::Field; + + use super::*; + use std::collections::HashMap; + use std::sync::Arc; + + #[test] + fn delta_log_schema_for_table_test() { + // NOTE: We should future proof the checkpoint schema in case action schema changes. + // See https://github.com/delta-io/delta-rs/issues/287 + + let table_schema = ArrowSchema::new(vec![ + ArrowField::new("pcol", ArrowDataType::Int32, true), + ArrowField::new("col1", ArrowDataType::Int32, true), + ]); + let partition_columns = vec!["pcol".to_string()]; + let log_schema = + delta_log_schema_for_table(table_schema.clone(), partition_columns.as_slice(), false); + + // verify top-level schema contains all expected fields and they are named correctly. + let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; + for f in log_schema.fields().iter() { + assert!(expected_fields.contains(&f.name().as_str())); + } + assert_eq!(5, log_schema.fields().len()); + + // verify add fields match as expected. a lot of transformation goes into these. + let add_fields: Vec<_> = log_schema + .fields() + .iter() + .filter(|f| f.name() == "add") + .flat_map(|f| { + if let ArrowDataType::Struct(fields) = f.data_type() { + fields.iter().cloned() + } else { + unreachable!(); + } + }) + .collect(); + let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); + assert_eq!( + vec![ + "path", + "size", + "modificationTime", + "dataChange", + "stats", + "partitionValues", + "tags", + "deletionVector", + "stats_parsed", + "partitionValues_parsed" + ], + field_names + ); + let add_field_map: HashMap<_, _> = add_fields + .iter() + .map(|f| (f.name().to_owned(), f.clone())) + .collect(); + let partition_values_parsed = add_field_map.get("partitionValues_parsed").unwrap(); + if let ArrowDataType::Struct(fields) = partition_values_parsed.data_type() { + assert_eq!(1, fields.len()); + let field = fields.get(0).unwrap().to_owned(); + assert_eq!( + Arc::new(ArrowField::new("pcol", ArrowDataType::Int32, true)), + field + ); + } else { + unreachable!(); + } + let stats_parsed = add_field_map.get("stats_parsed").unwrap(); + if let ArrowDataType::Struct(fields) = stats_parsed.data_type() { + assert_eq!(4, fields.len()); + + let field_map: HashMap<_, _> = fields + .iter() + .map(|f| (f.name().to_owned(), f.clone())) + .collect(); + + for (k, v) in field_map.iter() { + match k.as_ref() { + "minValues" | "maxValues" | "nullCount" => match v.data_type() { + ArrowDataType::Struct(fields) => { + assert_eq!(1, fields.len()); + let field = fields.get(0).unwrap().to_owned(); + let data_type = if k == "nullCount" { + ArrowDataType::Int64 + } else { + ArrowDataType::Int32 + }; + assert_eq!(Arc::new(ArrowField::new("col1", data_type, true)), field); + } + _ => unreachable!(), + }, + "numRecords" => {} + _ => panic!(), + } + } + } else { + unreachable!(); + } + + // verify extended remove schema fields **ARE NOT** included when `use_extended_remove_schema` is false. + let num_remove_fields = log_schema + .fields() + .iter() + .filter(|f| f.name() == "remove") + .flat_map(|f| { + if let ArrowDataType::Struct(fields) = f.data_type() { + fields.iter().cloned() + } else { + unreachable!(); + } + }) + .count(); + assert_eq!(4, num_remove_fields); + + // verify extended remove schema fields **ARE** included when `use_extended_remove_schema` is true. + let log_schema = + delta_log_schema_for_table(table_schema, partition_columns.as_slice(), true); + let remove_fields: Vec<_> = log_schema + .fields() + .iter() + .filter(|f| f.name() == "remove") + .flat_map(|f| { + if let ArrowDataType::Struct(fields) = f.data_type() { + fields.iter().cloned() + } else { + unreachable!(); + } + }) + .collect(); + assert_eq!(7, remove_fields.len()); + let expected_fields = [ + "path", + "deletionTimestamp", + "dataChange", + "extendedFileMetadata", + "partitionValues", + "size", + "tags", + ]; + for f in remove_fields.iter() { + assert!(expected_fields.contains(&f.name().as_str())); + } + } + + #[test] + fn test_arrow_from_delta_decimal_type() { + let precision = 20; + let scale = 2; + let decimal_field = DataType::Primitive(PrimitiveType::Decimal(precision, scale)); + assert_eq!( + >::try_from(&decimal_field).unwrap(), + ArrowDataType::Decimal128(precision as u8, scale as i8) + ); + } + + #[test] + fn test_arrow_from_delta_timestamp_type() { + let timestamp_field = DataType::Primitive(PrimitiveType::Timestamp); + assert_eq!( + >::try_from(×tamp_field).unwrap(), + ArrowDataType::Timestamp(TimeUnit::Microsecond, None) + ); + } + + #[test] + fn test_delta_from_arrow_timestamp_type() { + let timestamp_field = ArrowDataType::Timestamp(TimeUnit::Microsecond, None); + assert_eq!( + >::try_from(×tamp_field).unwrap(), + DataType::Primitive(PrimitiveType::Timestamp) + ); + } + + #[test] + fn test_delta_from_arrow_timestamp_type_with_tz() { + let timestamp_field = + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())); + assert_eq!( + >::try_from(×tamp_field).unwrap(), + DataType::Primitive(PrimitiveType::Timestamp) + ); + } + + #[test] + fn test_delta_from_arrow_map_type() { + let arrow_map = ArrowDataType::Map( + Arc::new(ArrowField::new( + "entries", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::Int8, false), + ArrowField::new("value", ArrowDataType::Binary, true), + ] + .into(), + ), + false, + )), + false, + ); + let converted_map: DataType = (&arrow_map).try_into().unwrap(); + + assert_eq!( + converted_map, + DataType::Map(Box::new(MapType::new( + DataType::Primitive(PrimitiveType::Byte), + DataType::Primitive(PrimitiveType::Binary), + true, + ))) + ); + } + + #[test] + fn test_record_batch_from_map_type() { + let keys = vec!["0", "1", "5", "6", "7"]; + let values: Vec<&[u8]> = vec![ + b"test_val_1", + b"test_val_2", + b"long_test_val_3", + b"4", + b"test_val_5", + ]; + let entry_offsets = vec![0u32, 1, 1, 4, 5, 5]; + let num_rows = keys.len(); + + // Copied the function `new_from_string` with the patched code from https://github.com/apache/arrow-rs/pull/4808 + // This should be reverted back [`MapArray::new_from_strings`] once arrow is upgraded in this project. + fn new_from_strings<'a>( + keys: impl Iterator, + values: &dyn Array, + entry_offsets: &[u32], + ) -> Result { + let entry_offsets_buffer = Buffer::from(entry_offsets.to_byte_slice()); + let keys_data = StringArray::from_iter_values(keys); + + let keys_field = Arc::new(Field::new("keys", ArrowDataType::Utf8, false)); + let values_field = Arc::new(Field::new( + "values", + values.data_type().clone(), + values.null_count() > 0, + )); + + let entry_struct = StructArray::from(vec![ + (keys_field, Arc::new(keys_data) as ArrayRef), + (values_field, make_array(values.to_data())), + ]); + + let map_data_type = ArrowDataType::Map( + Arc::new(Field::new( + "entries", + entry_struct.data_type().clone(), + false, + )), + false, + ); + + let map_data = ArrayData::builder(map_data_type) + .len(entry_offsets.len() - 1) + .add_buffer(entry_offsets_buffer) + .add_child_data(entry_struct.into_data()) + .build()?; + + Ok(MapArray::from(map_data)) + } + + let map_array = new_from_strings( + keys.into_iter(), + &arrow::array::BinaryArray::from(values), + entry_offsets.as_slice(), + ) + .expect("Could not create a map array"); + + let schema = + >::try_from(&StructType::new(vec![ + StructField::new( + "example".to_string(), + DataType::Map(Box::new(MapType::new( + DataType::Primitive(PrimitiveType::String), + DataType::Primitive(PrimitiveType::Binary), + false, + ))), + false, + ), + ])) + .expect("Could not get schema"); + + let record_batch = + arrow::record_batch::RecordBatch::try_new(Arc::new(schema), vec![Arc::new(map_array)]) + .expect("Failed to create RecordBatch"); + + assert_eq!(record_batch.num_columns(), 1); + assert_eq!(record_batch.num_rows(), num_rows); + } + + #[test] + fn test_max_min_schema_for_fields() { + let mut max_min_vec: Vec = Vec::new(); + let fields = [ + ArrowField::new("simple", ArrowDataType::Int32, true), + ArrowField::new( + "struct", + ArrowDataType::Struct( + vec![ArrowField::new("simple", ArrowDataType::Int32, true)].into(), + ), + true, + ), + ArrowField::new( + "list", + ArrowDataType::List(Arc::new(ArrowField::new( + "simple", + ArrowDataType::Int32, + true, + ))), + true, + ), + ArrowField::new( + "map", + ArrowDataType::Map( + Arc::new(ArrowField::new( + "struct", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::Int32, true), + ArrowField::new("value", ArrowDataType::Int32, true), + ] + .into(), + ), + true, + )), + true, + ), + true, + ), + ArrowField::new("binary", ArrowDataType::Binary, true), + ]; + + let expected = vec![fields[0].clone(), fields[1].clone()]; + + fields + .iter() + .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); + + assert_eq!(max_min_vec, expected); + } + + #[test] + fn test_null_count_schema_for_fields() { + let mut null_count_vec: Vec = Vec::new(); + let fields = [ + ArrowField::new("int32", ArrowDataType::Int32, true), + ArrowField::new("int64", ArrowDataType::Int64, true), + ArrowField::new("Utf8", ArrowDataType::Utf8, true), + ArrowField::new( + "list", + ArrowDataType::List(Arc::new(ArrowField::new( + "simple", + ArrowDataType::Int32, + true, + ))), + true, + ), + ArrowField::new( + "map", + ArrowDataType::Map( + Arc::new(ArrowField::new( + "struct", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::Int32, true), + ArrowField::new("value", ArrowDataType::Int32, true), + ] + .into(), + ), + true, + )), + true, + ), + true, + ), + ArrowField::new( + "struct", + ArrowDataType::Struct( + vec![ArrowField::new("int32", ArrowDataType::Int32, true)].into(), + ), + true, + ), + ]; + let expected = vec![ + ArrowField::new(fields[0].name(), ArrowDataType::Int64, true), + ArrowField::new(fields[1].name(), ArrowDataType::Int64, true), + ArrowField::new(fields[2].name(), ArrowDataType::Int64, true), + ArrowField::new(fields[3].name(), ArrowDataType::Int64, true), + ArrowField::new(fields[4].name(), ArrowDataType::Int64, true), + ArrowField::new( + fields[5].name(), + ArrowDataType::Struct( + vec![ArrowField::new("int32", ArrowDataType::Int64, true)].into(), + ), + true, + ), + ]; + fields + .iter() + .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); + assert_eq!(null_count_vec, expected); + } + + /* + * This test validates the trait implementation of + * TryFrom<&Arc> for schema::SchemaField which is required with Arrow 37 since + * iterators on Fields will give an &Arc + */ + #[test] + fn tryfrom_arrowfieldref_with_structs() { + let field = Arc::new(ArrowField::new( + "test_struct", + ArrowDataType::Struct( + vec![ + ArrowField::new("key", ArrowDataType::Int32, true), + ArrowField::new("value", ArrowDataType::Int32, true), + ] + .into(), + ), + true, + )); + let _converted: StructField = field.as_ref().try_into().unwrap(); + } +} diff --git a/crates/deltalake-core/src/table/config.rs b/crates/deltalake-core/src/table/config.rs index 5b82b401b6..3fa021ce6e 100644 --- a/crates/deltalake-core/src/table/config.rs +++ b/crates/deltalake-core/src/table/config.rs @@ -11,7 +11,6 @@ use crate::errors::DeltaTableError; /// /// #[derive(PartialEq, Eq, Hash)] -#[non_exhaustive] pub enum DeltaConfigKey { /// true for this Delta table to be append-only. If append-only, /// existing records cannot be deleted, and existing values cannot be updated. @@ -101,9 +100,6 @@ pub enum DeltaConfigKey { /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. TuneFileSizesForRewrites, - - /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. - CheckpointPolicy, } impl AsRef for DeltaConfigKey { @@ -115,7 +111,6 @@ impl AsRef for DeltaConfigKey { Self::AutoOptimizeOptimizeWrite => "delta.autoOptimize.optimizeWrite", Self::CheckpointWriteStatsAsJson => "delta.checkpoint.writeStatsAsJson", Self::CheckpointWriteStatsAsStruct => "delta.checkpoint.writeStatsAsStruct", - Self::CheckpointPolicy => "delta.checkpointPolicy", Self::ColumnMappingMode => "delta.columnMapping.mode", Self::DataSkippingNumIndexedCols => "delta.dataSkippingNumIndexedCols", Self::DeletedFileRetentionDuration => "delta.deletedFileRetentionDuration", @@ -145,7 +140,6 @@ impl FromStr for DeltaConfigKey { "delta.autoOptimize.optimizeWrite" => Ok(Self::AutoOptimizeOptimizeWrite), "delta.checkpoint.writeStatsAsJson" => Ok(Self::CheckpointWriteStatsAsJson), "delta.checkpoint.writeStatsAsStruct" => Ok(Self::CheckpointWriteStatsAsStruct), - "delta.checkpointPolicy" => Ok(Self::CheckpointPolicy), "delta.columnMapping.mode" => Ok(Self::ColumnMappingMode), "delta.dataSkippingNumIndexedCols" => Ok(Self::DataSkippingNumIndexedCols), "delta.deletedFileRetentionDuration" | "deletedFileRetentionDuration" => { @@ -286,14 +280,6 @@ impl<'a> TableConfig<'a> { .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) .unwrap_or_default() } - - /// Policy applied during chepoint creation - pub fn checkpoint_policy(&self) -> CheckpointPolicy { - self.0 - .get(DeltaConfigKey::CheckpointPolicy.as_ref()) - .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) - .unwrap_or_default() - } } #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] @@ -352,48 +338,6 @@ impl FromStr for IsolationLevel { } } -#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] -/// The checkpoint policy applied when writing checkpoints -#[serde(rename_all = "camelCase")] -pub enum CheckpointPolicy { - /// classic Delta Lake checkpoints - Classic, - /// v2 checkpoints - V2, - /// unknown checkpoint policy - Other(String), -} - -impl Default for CheckpointPolicy { - fn default() -> Self { - Self::Classic - } -} - -impl AsRef for CheckpointPolicy { - fn as_ref(&self) -> &str { - match self { - Self::Classic => "classic", - Self::V2 => "v2", - Self::Other(s) => s, - } - } -} - -impl FromStr for CheckpointPolicy { - type Err = DeltaTableError; - - fn from_str(s: &str) -> Result { - match s.to_ascii_lowercase().as_str() { - "classic" => Ok(Self::Classic), - "v2" => Ok(Self::V2), - _ => Err(DeltaTableError::Generic( - "Invalid string for CheckpointPolicy".into(), - )), - } - } -} - const SECONDS_PER_MINUTE: u64 = 60; const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; diff --git a/python/deltalake/__init__.py b/python/deltalake/__init__.py index b10a708309..129eaff1cf 100644 --- a/python/deltalake/__init__.py +++ b/python/deltalake/__init__.py @@ -6,5 +6,4 @@ from .schema import Schema as Schema from .table import DeltaTable as DeltaTable from .table import Metadata as Metadata -from .writer import convert_to_deltalake as convert_to_deltalake from .writer import write_deltalake as write_deltalake diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 854d96d4a8..2a4f77993c 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -152,16 +152,6 @@ def write_to_deltalake( configuration: Optional[Mapping[str, Optional[str]]], storage_options: Optional[Dict[str, str]], ) -> None: ... -def convert_to_deltalake( - uri: str, - partition_by: Optional[pyarrow.Schema], - partition_strategy: Optional[Literal["hive"]], - name: Optional[str], - description: Optional[str], - configuration: Optional[Mapping[str, Optional[str]]], - storage_options: Optional[Dict[str, str]], - custom_metadata: Optional[Dict[str, str]], -) -> None: ... def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... # Can't implement inheritance (see note in src/schema.rs), so this is next diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index d4b55368d5..01b7382751 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -38,7 +38,6 @@ from ._internal import DeltaDataChecker as _DeltaDataChecker from ._internal import batch_distinct -from ._internal import convert_to_deltalake as _convert_to_deltalake from ._internal import write_new_deltalake as write_deltalake_pyarrow from ._internal import write_to_deltalake as write_deltalake_rust from .exceptions import DeltaProtocolError, TableNotFoundError @@ -444,60 +443,6 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: raise ValueError("Only `pyarrow` or `rust` are valid inputs for the engine.") -def convert_to_deltalake( - uri: Union[str, Path], - mode: Literal["error", "ignore"] = "error", - partition_by: Optional[pa.Schema] = None, - partition_strategy: Optional[Literal["hive"]] = None, - name: Optional[str] = None, - description: Optional[str] = None, - configuration: Optional[Mapping[str, Optional[str]]] = None, - storage_options: Optional[Dict[str, str]] = None, - custom_metadata: Optional[Dict[str, str]] = None, -) -> None: - """ - `Convert` parquet tables `to delta` tables. - - Currently only HIVE partitioned tables are supported. `Convert to delta` creates - a transaction log commit with add actions, and additional properties provided such - as configuration, name, and description. - - Args: - uri: URI of a table. - partition_by: Optional partitioning schema if table is partitioned. - partition_strategy: Optional partition strategy to read and convert - mode: How to handle existing data. Default is to error if table already exists. - If 'ignore', will not convert anything if table already exists. - name: User-provided identifier for this table. - description: User-provided description for this table. - configuration: A map containing configuration options for the metadata action. - storage_options: options passed to the native delta filesystem. Unused if 'filesystem' is defined. - custom_metadata: custom metadata that will be added to the transaction commit - """ - if partition_by is not None and partition_strategy is None: - raise ValueError("Partition strategy has to be provided with partition_by.") - - if partition_strategy is not None and partition_strategy != "hive": - raise ValueError( - "Currently only `hive` partition strategy is supported to be converted." - ) - - if mode == "ignore" and try_get_deltatable(uri, storage_options) is not None: - return - - _convert_to_deltalake( - str(uri), - partition_by, - partition_strategy, - name, - description, - configuration, - storage_options, - custom_metadata, - ) - return - - def __enforce_append_only( table: Optional[DeltaTable], configuration: Optional[Mapping[str, Optional[str]]], diff --git a/python/src/lib.rs b/python/src/lib.rs index 86bbe1f16c..eddd742951 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -27,7 +27,6 @@ use deltalake::datafusion::prelude::SessionContext; use deltalake::delta_datafusion::DeltaDataChecker; use deltalake::errors::DeltaTableError; use deltalake::kernel::{Action, Add, Invariant, Metadata, Remove, StructType}; -use deltalake::operations::convert_to_delta::{ConvertToDeltaBuilder, PartitionStrategy}; use deltalake::operations::delete::DeleteBuilder; use deltalake::operations::filesystem_check::FileSystemCheckBuilder; use deltalake::operations::merge::MergeBuilder; @@ -44,7 +43,6 @@ use deltalake::DeltaTableBuilder; use pyo3::exceptions::{PyIOError, PyRuntimeError, PyValueError}; use pyo3::prelude::*; use pyo3::types::{PyFrozenSet, PyType}; -use serde_json::{Map, Value}; use crate::error::DeltaProtocolError; use crate::error::PythonError; @@ -760,8 +758,7 @@ impl RawDeltaTable { schema: PyArrowType, partitions_filters: Option>, ) -> PyResult<()> { - let mode = mode.parse().map_err(PythonError::from)?; - + let mode = save_mode_from_str(mode)?; let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; let existing_schema = self._table.get_schema().map_err(PythonError::from)?; @@ -1091,6 +1088,16 @@ fn batch_distinct(batch: PyArrowType) -> PyResult PyResult { + match value { + "append" => Ok(SaveMode::Append), + "overwrite" => Ok(SaveMode::Overwrite), + "error" => Ok(SaveMode::ErrorIfExists), + "ignore" => Ok(SaveMode::Ignore), + _ => Err(PyValueError::new_err("Invalid save mode")), + } +} + fn current_timestamp() -> i64 { let start = SystemTime::now(); let since_the_epoch = start @@ -1226,58 +1233,6 @@ fn write_new_deltalake( Ok(()) } -#[pyfunction] -#[allow(clippy::too_many_arguments)] -fn convert_to_deltalake( - uri: String, - partition_schema: Option>, - partition_strategy: Option, - name: Option, - description: Option, - configuration: Option>>, - storage_options: Option>, - custom_metadata: Option>, -) -> PyResult<()> { - let mut builder = ConvertToDeltaBuilder::new().with_location(uri); - - if let Some(part_schema) = partition_schema { - let schema: StructType = (&part_schema.0).try_into().map_err(PythonError::from)?; - builder = builder.with_partition_schema(schema.fields().clone()); - } - - if let Some(partition_strategy) = &partition_strategy { - let strategy: PartitionStrategy = partition_strategy.parse().map_err(PythonError::from)?; - builder = builder.with_partition_strategy(strategy); - } - - if let Some(name) = &name { - builder = builder.with_table_name(name); - } - - if let Some(description) = &description { - builder = builder.with_comment(description); - } - - if let Some(config) = configuration { - builder = builder.with_configuration(config); - }; - - if let Some(strg_options) = storage_options { - builder = builder.with_storage_options(strg_options); - }; - - if let Some(metadata) = custom_metadata { - let json_metadata: Map = - metadata.into_iter().map(|(k, v)| (k, v.into())).collect(); - builder = builder.with_metadata(json_metadata); - }; - - rt()? - .block_on(builder.into_future()) - .map_err(PythonError::from)?; - Ok(()) -} - #[pyclass(name = "DeltaDataChecker", module = "deltalake._internal")] struct PyDeltaDataChecker { inner: DeltaDataChecker, @@ -1324,7 +1279,6 @@ fn _internal(py: Python, m: &PyModule) -> PyResult<()> { m.add_function(pyo3::wrap_pyfunction!(rust_core_version, m)?)?; m.add_function(pyo3::wrap_pyfunction!(write_new_deltalake, m)?)?; m.add_function(pyo3::wrap_pyfunction!(write_to_deltalake, m)?)?; - m.add_function(pyo3::wrap_pyfunction!(convert_to_deltalake, m)?)?; m.add_function(pyo3::wrap_pyfunction!(batch_distinct, m)?)?; m.add_class::()?; m.add_class::()?; diff --git a/python/tests/test_convert_to_delta.py b/python/tests/test_convert_to_delta.py deleted file mode 100644 index 29badf3358..0000000000 --- a/python/tests/test_convert_to_delta.py +++ /dev/null @@ -1,97 +0,0 @@ -import pathlib - -import pyarrow as pa -import pyarrow.dataset as ds -import pytest - -from deltalake import convert_to_deltalake -from deltalake.exceptions import DeltaError -from deltalake.table import DeltaTable - - -def test_local_convert_to_delta(tmp_path: pathlib.Path, sample_data: pa.Table): - ds.write_dataset( - sample_data, - tmp_path, - format="parquet", - existing_data_behavior="overwrite_or_ignore", - ) - - name = "converted_table" - description = "parquet table converted to delta table with delta-rs" - convert_to_deltalake( - tmp_path, - name=name, - description=description, - configuration={"delta.AppendOnly": "True"}, - ) - - dt = DeltaTable(tmp_path) - - assert dt.version() == 0 - assert dt.files() == ["part-0.parquet"] - assert dt.metadata().name == name - assert dt.metadata().description == description - assert dt.metadata().configuration == {"delta.AppendOnly": "True"} - - -def test_convert_delta_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): - ds.write_dataset( - sample_data, - tmp_path, - format="parquet", - existing_data_behavior="overwrite_or_ignore", - ) - - convert_to_deltalake( - tmp_path, - ) - - with pytest.raises(DeltaError): - convert_to_deltalake( - tmp_path, - ) - - convert_to_deltalake(tmp_path, mode="ignore") - - -def test_convert_delta_with_partitioning(tmp_path: pathlib.Path, sample_data: pa.Table): - ds.write_dataset( - sample_data, - tmp_path, - format="parquet", - existing_data_behavior="overwrite_or_ignore", - partitioning=["utf8"], - partitioning_flavor="hive", - ) - - with pytest.raises( - DeltaError, - match="Generic error: The schema of partition columns must be provided to convert a Parquet table to a Delta table", - ): - convert_to_deltalake( - tmp_path, - ) - with pytest.raises( - ValueError, match="Partition strategy has to be provided with partition_by" - ): - convert_to_deltalake( - tmp_path, - partition_by=pa.schema([pa.field("utf8", pa.string())]), - ) - - with pytest.raises( - ValueError, - match="Currently only `hive` partition strategy is supported to be converted.", - ): - convert_to_deltalake( - tmp_path, - partition_by=pa.schema([pa.field("utf8", pa.string())]), - partition_strategy="directory", - ) - - convert_to_deltalake( - tmp_path, - partition_by=pa.schema([pa.field("utf8", pa.string())]), - partition_strategy="hive", - ) From 3e2556143654b58de3c2b59d3359202e3d0c1312 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Mon, 20 Nov 2023 21:53:53 +0100 Subject: [PATCH 19/35] formatting --- python/deltalake/writer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index a03e9f4b59..d4b55368d5 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -38,9 +38,9 @@ from ._internal import DeltaDataChecker as _DeltaDataChecker from ._internal import batch_distinct +from ._internal import convert_to_deltalake as _convert_to_deltalake from ._internal import write_new_deltalake as write_deltalake_pyarrow from ._internal import write_to_deltalake as write_deltalake_rust -from ._internal import convert_to_deltalake as _convert_to_deltalake from .exceptions import DeltaProtocolError, TableNotFoundError from .table import MAX_SUPPORTED_WRITER_VERSION, DeltaTable From d9a4ce023e2fc3b6e2836a55498d8dae5e412407 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Mon, 20 Nov 2023 22:04:17 +0100 Subject: [PATCH 20/35] use fromstr --- python/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/src/lib.rs b/python/src/lib.rs index 86bbe1f16c..a2d0161d5a 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1143,7 +1143,7 @@ fn write_to_deltalake( storage_options: Option>, ) -> PyResult<()> { let batches = data.0.map(|batch| batch.unwrap()).collect::>(); - let save_mode = save_mode_from_str(&mode)?; + let save_mode = mode.parse().map_err(PythonError::from)?; let options = storage_options.clone().unwrap_or_default(); let table = rt()? From e3c71894faf266b29555db8a2b1263eb9c8c2477 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Mon, 20 Nov 2023 22:37:47 +0100 Subject: [PATCH 21/35] add overwrite_schema support --- .../deltalake-core/src/operations/delete.rs | 1 + crates/deltalake-core/src/operations/merge.rs | 1 + .../deltalake-core/src/operations/update.rs | 1 + crates/deltalake-core/src/operations/write.rs | 36 ++++++++++++++++--- python/deltalake/writer.py | 5 --- python/tests/test_writer.py | 27 ++++++++++---- 6 files changed, 55 insertions(+), 16 deletions(-) diff --git a/crates/deltalake-core/src/operations/delete.rs b/crates/deltalake-core/src/operations/delete.rs index b6c94f423b..35cf0b858f 100644 --- a/crates/deltalake-core/src/operations/delete.rs +++ b/crates/deltalake-core/src/operations/delete.rs @@ -172,6 +172,7 @@ async fn excute_non_empty_expr( None, writer_properties, false, + false, ) .await?; diff --git a/crates/deltalake-core/src/operations/merge.rs b/crates/deltalake-core/src/operations/merge.rs index 8b0dd56708..061c2eb912 100644 --- a/crates/deltalake-core/src/operations/merge.rs +++ b/crates/deltalake-core/src/operations/merge.rs @@ -1013,6 +1013,7 @@ async fn execute( None, writer_properties, safe_cast, + false, ) .await?; diff --git a/crates/deltalake-core/src/operations/update.rs b/crates/deltalake-core/src/operations/update.rs index 907dec5998..fa44724beb 100644 --- a/crates/deltalake-core/src/operations/update.rs +++ b/crates/deltalake-core/src/operations/update.rs @@ -363,6 +363,7 @@ async fn execute( None, writer_properties, safe_cast, + false, ) .await?; diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index 5230fd09dd..c868048b99 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -43,7 +43,7 @@ use super::writer::{DeltaWriter, WriterConfig}; use super::{transaction::commit, CreateBuilder}; use crate::delta_datafusion::DeltaDataChecker; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Action, Add, Remove, StructType}; +use crate::kernel::{Action, Add, Metadata, Remove, StructType}; use crate::logstore::LogStoreRef; use crate::protocol::{DeltaOperation, SaveMode}; use crate::storage::ObjectStoreRef; @@ -304,6 +304,7 @@ pub(crate) async fn write_execution_plan( write_batch_size: Option, writer_properties: Option, safe_cast: bool, + overwrite_schema: bool, ) -> DeltaResult> { let invariants = snapshot .current_metadata() @@ -311,7 +312,11 @@ pub(crate) async fn write_execution_plan( .unwrap_or_default(); // Use input schema to prevent wrapping partitions columns into a dictionary. - let schema = snapshot.input_schema().unwrap_or(plan.schema()); + let schema: ArrowSchemaRef = if overwrite_schema { + plan.schema() + } else { + snapshot.input_schema().unwrap_or(plan.schema()) + }; let checker = DeltaDataChecker::new(invariants); @@ -392,13 +397,14 @@ impl std::future::IntoFuture for WriteBuilder { Ok(this.partition_columns.unwrap_or_default()) }?; + let mut schema: ArrowSchemaRef = arrow_schema::Schema::empty().into(); let plan = if let Some(plan) = this.input { Ok(plan) } else if let Some(batches) = this.batches { if batches.is_empty() { Err(WriteError::MissingData) } else { - let schema = batches[0].schema(); + schema = batches[0].schema(); let table_schema = this .snapshot .physical_arrow_schema(this.log_store.object_store().clone()) @@ -407,7 +413,7 @@ impl std::future::IntoFuture for WriteBuilder { .unwrap_or(schema.clone()); if !can_cast_batch(schema.fields(), table_schema.fields()) - && !this.overwrite_schema + && !(this.overwrite_schema && matches!(this.mode, SaveMode::Overwrite)) { return Err(DeltaTableError::Generic( "Schema of data does not match table schema".to_string(), @@ -445,7 +451,7 @@ impl std::future::IntoFuture for WriteBuilder { vec![batches] }; - Ok(Arc::new(MemoryExec::try_new(&data, schema, None)?) + Ok(Arc::new(MemoryExec::try_new(&data, schema.clone(), None)?) as Arc) } } else { @@ -470,12 +476,32 @@ impl std::future::IntoFuture for WriteBuilder { this.write_batch_size, this.writer_properties, this.safe_cast, + this.overwrite_schema, ) .await?; actions.extend(add_actions.into_iter().map(Action::Add)); // Collect remove actions if we are overwriting the table if matches!(this.mode, SaveMode::Overwrite) { + // Update metadata with new schema + let table_schema = this + .snapshot + .physical_arrow_schema(this.log_store.object_store().clone()) + .await + .or_else(|_| this.snapshot.arrow_schema()) + .unwrap_or(schema.clone()); + + // let data_schema: StructType = schema.clone().try_into().unwrap(); + if schema != table_schema { + let mut metadata = this + .snapshot + .current_metadata() + .ok_or(DeltaTableError::NoMetadata)? + .clone(); + metadata.schema = schema.clone().try_into().unwrap(); + let metadata_action = Metadata::try_from(metadata).unwrap(); + actions.push(Action::Metadata(metadata_action)); + } // This should never error, since now() will always be larger than UNIX_EPOCH let deletion_timestamp = SystemTime::now() .duration_since(UNIX_EPOCH) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index d4b55368d5..358f7fed41 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -185,11 +185,6 @@ def write_deltalake( if table is not None and mode == "ignore": return - if mode == "overwrite" and overwrite_schema: - raise NotImplementedError( - "The rust engine writer does not yet support schema evolution." - ) - if isinstance(data, RecordBatchReader): batch_iter = data elif isinstance(data, pa.RecordBatch): diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 808a2f9563..3de0bc2749 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -156,19 +156,34 @@ def test_update_schema_rust_writer(existing_table: DeltaTable): overwrite_schema=True, engine="rust", ) - # TODO(ion): Remove this once we add schema overwrite support - with pytest.raises(NotImplementedError): + with pytest.raises(DeltaError): write_deltalake( existing_table, new_data, mode="overwrite", - overwrite_schema=True, + overwrite_schema=False, engine="rust", ) + with pytest.raises(DeltaError): + write_deltalake( + existing_table, + new_data, + mode="append", + overwrite_schema=False, + engine="rust", + ) + # TODO(ion): Remove this once we add schema overwrite support + write_deltalake( + existing_table, + new_data, + mode="overwrite", + overwrite_schema=True, + engine="rust", + ) - read_data = existing_table.to_pyarrow_table() - assert new_data == read_data - assert existing_table.schema().to_pyarrow() == new_data.schema + read_data = existing_table.to_pyarrow_table() + assert new_data == read_data + assert existing_table.schema().to_pyarrow() == new_data.schema @pytest.mark.parametrize("engine", ["pyarrow", "rust"]) From 5af1251fc00a16ef714229fe43c4ee92e07d1fa9 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Mon, 20 Nov 2023 23:02:15 +0100 Subject: [PATCH 22/35] fix clippy of unrelated code ? --- crates/deltalake-core/src/lib.rs | 1 + crates/deltalake-core/src/protocol/checkpoints.rs | 7 ++----- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/crates/deltalake-core/src/lib.rs b/crates/deltalake-core/src/lib.rs index 644da2dcac..97047f7880 100644 --- a/crates/deltalake-core/src/lib.rs +++ b/crates/deltalake-core/src/lib.rs @@ -71,6 +71,7 @@ #![deny(warnings)] #![deny(missing_docs)] #![allow(rustdoc::invalid_html_tags)] +#![allow(clippy::nonminimal_bool)] #[cfg(all(feature = "parquet", feature = "parquet2"))] compile_error!( diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs index 837483c35c..37aa04012a 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -468,9 +468,8 @@ fn apply_stats_conversion( data_type: &DataType, ) { if path.len() == 1 { - match data_type { - DataType::Primitive(PrimitiveType::Timestamp) => { - let v = context.get_mut(&path[0]); + if let DataType::Primitive(PrimitiveType::Timestamp) = data_type { + let v = context.get_mut(&path[0]); if let Some(v) = v { let ts = v @@ -483,8 +482,6 @@ fn apply_stats_conversion( } } } - _ => { /* noop */ } - } } else { let next_context = context.get_mut(&path[0]).and_then(|v| v.as_object_mut()); if let Some(next_context) = next_context { From 37443843b5926fb0fa4e81ae6401876e7afa3fc8 Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Mon, 20 Nov 2023 23:03:32 +0100 Subject: [PATCH 23/35] cargo fmt --- .../deltalake-core/src/protocol/checkpoints.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs index 37aa04012a..ef521159d9 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -469,19 +469,19 @@ fn apply_stats_conversion( ) { if path.len() == 1 { if let DataType::Primitive(PrimitiveType::Timestamp) = data_type { - let v = context.get_mut(&path[0]); + let v = context.get_mut(&path[0]); - if let Some(v) = v { - let ts = v - .as_str() - .and_then(|s| time_utils::timestamp_micros_from_stats_string(s).ok()) - .map(|n| Value::Number(serde_json::Number::from(n))); + if let Some(v) = v { + let ts = v + .as_str() + .and_then(|s| time_utils::timestamp_micros_from_stats_string(s).ok()) + .map(|n| Value::Number(serde_json::Number::from(n))); - if let Some(ts) = ts { - *v = ts; - } + if let Some(ts) = ts { + *v = ts; } } + } } else { let next_context = context.get_mut(&path[0]).and_then(|v| v.as_object_mut()); if let Some(next_context) = next_context { From 2e1f0c98f66cdbc99fb77e93bd4bae3199b2b2cf Mon Sep 17 00:00:00 2001 From: Nikolay Ulmasov Date: Sun, 19 Nov 2023 16:20:00 +0000 Subject: [PATCH 24/35] resolve #1860 Signed-off-by: Nikolay Ulmasov --- crates/deltalake-core/src/data_catalog/mod.rs | 12 ++++++------ crates/deltalake-core/src/lib.rs | 5 +++++ python/src/lib.rs | 4 +--- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/crates/deltalake-core/src/data_catalog/mod.rs b/crates/deltalake-core/src/data_catalog/mod.rs index 911d4292f8..db05e429bb 100644 --- a/crates/deltalake-core/src/data_catalog/mod.rs +++ b/crates/deltalake-core/src/data_catalog/mod.rs @@ -7,7 +7,7 @@ pub use unity::*; #[cfg(feature = "unity-experimental")] pub mod client; -#[cfg(feature = "glue")] +#[cfg(any(feature = "glue", feature = "glue-native-tls"))] pub mod glue; #[cfg(feature = "datafusion")] pub mod storage; @@ -49,7 +49,7 @@ pub enum DataCatalogError { }, /// Missing metadata in the catalog - #[cfg(feature = "glue")] + #[cfg(any(feature = "glue", feature = "glue-native-tls"))] #[error("Missing Metadata {metadata} in the Data Catalog ")] MissingMetadata { /// The missing metadata property @@ -57,7 +57,7 @@ pub enum DataCatalogError { }, /// Glue Glue Data Catalog Error - #[cfg(feature = "glue")] + #[cfg(any(feature = "glue", feature = "glue-native-tls"))] #[error("Catalog glue error: {source}")] GlueError { /// The underlying Glue Data Catalog Error @@ -66,7 +66,7 @@ pub enum DataCatalogError { }, /// Error caused by the http request dispatcher not being able to be created. - #[cfg(feature = "glue")] + #[cfg(any(feature = "glue", feature = "glue-native-tls"))] #[error("Failed to create request dispatcher: {source}")] AWSHttpClient { /// The underlying Rusoto TlsError @@ -75,7 +75,7 @@ pub enum DataCatalogError { }, /// Error representing a failure to retrieve AWS credentials. - #[cfg(feature = "glue")] + #[cfg(any(feature = "glue", feature = "glue-native-tls"))] #[error("Failed to retrieve AWS credentials: {source}")] AWSCredentials { /// The underlying Rusoto CredentialsError @@ -138,7 +138,7 @@ pub fn get_data_catalog( "azure" => unimplemented!("Azure Data Catalog is not implemented"), #[cfg(feature = "hdfs")] "hdfs" => unimplemented!("HDFS Data Catalog is not implemented"), - #[cfg(feature = "glue")] + #[cfg(any(feature = "glue", feature = "glue-native-tls"))] "glue" => Ok(Box::new(glue::GlueDataCatalog::new()?)), #[cfg(feature = "unity-experimental")] "unity" => { diff --git a/crates/deltalake-core/src/lib.rs b/crates/deltalake-core/src/lib.rs index 97047f7880..6425ec3282 100644 --- a/crates/deltalake-core/src/lib.rs +++ b/crates/deltalake-core/src/lib.rs @@ -83,6 +83,11 @@ compile_error!( "Features s3 and s3-native-tls are mutually exclusive and cannot be enabled together" ); +#[cfg(all(feature = "glue", feature = "glue-native-tls"))] +compile_error!( + "Features glue and glue-native-tls are mutually exclusive and cannot be enabled together" +); + pub mod data_catalog; pub mod errors; pub mod kernel; diff --git a/python/src/lib.rs b/python/src/lib.rs index a2d0161d5a..be0550ff4d 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -134,9 +134,7 @@ impl RawDeltaTable { catalog_options: Option>, ) -> PyResult { let data_catalog = deltalake::data_catalog::get_data_catalog(data_catalog, catalog_options) - .map_err(|_| { - PyValueError::new_err(format!("Catalog '{}' not available.", data_catalog)) - })?; + .map_err(|e| PyValueError::new_err(format!("{}", e)))?; let table_uri = rt()? .block_on(data_catalog.get_table_storage_location( data_catalog_id, From fee4d771f142f9a625e857951fa46df21de721f6 Mon Sep 17 00:00:00 2001 From: Matthew Powers Date: Wed, 22 Nov 2023 15:23:14 -0500 Subject: [PATCH 25/35] docs: on append, overwrite, delete and z-ordering (#1897) # Description Adds docs on how to append, overwrite, delete rows, and Z Order Delta tables. Will add much more detailed pages in the future. Just getting the high-level skeleton of the docs developed. --- .../appending-overwriting-delta-lake-table.md | 78 +++++++++++++++++++ docs/usage/create-delta-lake-table.md | 25 ++++++ .../deleting-rows-from-delta-lake-table.md | 34 ++++++++ docs/usage/optimize/delta-lake-z-order.md | 16 ++++ .../small-file-compaction-with-optimize.md | 0 mkdocs.yml | 17 ++-- 6 files changed, 164 insertions(+), 6 deletions(-) create mode 100644 docs/usage/appending-overwriting-delta-lake-table.md create mode 100644 docs/usage/create-delta-lake-table.md create mode 100644 docs/usage/deleting-rows-from-delta-lake-table.md create mode 100644 docs/usage/optimize/delta-lake-z-order.md rename docs/usage/{ => optimize}/small-file-compaction-with-optimize.md (100%) diff --git a/docs/usage/appending-overwriting-delta-lake-table.md b/docs/usage/appending-overwriting-delta-lake-table.md new file mode 100644 index 0000000000..0930d8da1e --- /dev/null +++ b/docs/usage/appending-overwriting-delta-lake-table.md @@ -0,0 +1,78 @@ +# Appending to and overwriting a Delta Lake table + +This section explains how to append to an exising Delta table and how to overwrite a Delta table. + +## Delta Lake append transactions + +Suppose you have a Delta table with the following contents: + +``` ++-------+----------+ +| num | letter | +|-------+----------| +| 1 | a | +| 2 | b | +| 3 | c | ++-------+----------+ +``` + +Append two additional rows of data to the table: + +```python +from deltalake import write_deltalake, DeltaTable + +df = pd.DataFrame({"num": [8, 9], "letter": ["dd", "ee"]}) +write_deltalake("tmp/some-table", df, mode="append") +``` + +Here are the updated contents of the Delta table: + +``` ++-------+----------+ +| num | letter | +|-------+----------| +| 1 | a | +| 2 | b | +| 3 | c | +| 8 | dd | +| 9 | ee | ++-------+----------+ +``` + +Now let's see how to perform an overwrite transaction. + +## Delta Lake overwrite transactions + +Now let's see how to overwrite the exisitng Delta table. + +```python +df = pd.DataFrame({"num": [11, 22], "letter": ["aa", "bb"]}) +write_deltalake("tmp/some-table", df, mode="overwrite") +``` + +Here are the contents of the Delta table after the overwrite operation: + +``` ++-------+----------+ +| num | letter | +|-------+----------| +| 11 | aa | +| 22 | bb | ++-------+----------+ +``` + +Overwriting just performs a logical delete. It doesn't physically remove the previous data from storage. Time travel back to the previous version to confirm that the old version of the table is still accessable. + +``` +dt = DeltaTable("tmp/some-table", version=1) + ++-------+----------+ +| num | letter | +|-------+----------| +| 1 | a | +| 2 | b | +| 3 | c | +| 8 | dd | +| 9 | ee | ++-------+----------+ +``` diff --git a/docs/usage/create-delta-lake-table.md b/docs/usage/create-delta-lake-table.md new file mode 100644 index 0000000000..3a2f023a47 --- /dev/null +++ b/docs/usage/create-delta-lake-table.md @@ -0,0 +1,25 @@ +# Creating a Delta Lake Table + +This section explains how to create a Delta Lake table. + +You can easily write a DataFrame to a Delta table. + +```python +from deltalake import write_deltalake +import pandas as pd + +df = pd.DataFrame({"num": [1, 2, 3], "letter": ["a", "b", "c"]}) +write_deltalake("tmp/some-table", df) +``` + +Here are the contents of the Delta table in storage: + +``` ++-------+----------+ +| num | letter | +|-------+----------| +| 1 | a | +| 2 | b | +| 3 | c | ++-------+----------+ +``` diff --git a/docs/usage/deleting-rows-from-delta-lake-table.md b/docs/usage/deleting-rows-from-delta-lake-table.md new file mode 100644 index 0000000000..e1833c84b9 --- /dev/null +++ b/docs/usage/deleting-rows-from-delta-lake-table.md @@ -0,0 +1,34 @@ +# Deleting rows from a Delta Lake table + +This section explains how to delete rows from a Delta Lake table. + +Suppose you have the following Delta table with four rows: + +``` ++-------+----------+ +| num | letter | +|-------+----------| +| 1 | a | +| 2 | b | +| 3 | c | +| 4 | d | ++-------+----------+ +``` + +Here's how to delete all the rows where the `num` is greater than 2: + +```python +dt = DeltaTable("tmp/my-table") +dt.delete("num > 2") +``` + +Here are the contents of the Delta table after the delete operation has been performed: + +``` ++-------+----------+ +| num | letter | +|-------+----------| +| 1 | a | +| 2 | b | ++-------+----------+ +``` diff --git a/docs/usage/optimize/delta-lake-z-order.md b/docs/usage/optimize/delta-lake-z-order.md new file mode 100644 index 0000000000..54be212c47 --- /dev/null +++ b/docs/usage/optimize/delta-lake-z-order.md @@ -0,0 +1,16 @@ +# Delta Lake Z Order + +This section explains how to Z Order a Delta table. + +Z Ordering colocates similar data in the same files, which allows for better file skipping and faster queries. + +Suppose you have a table with `first_name`, `age`, and `country` columns. + +If you Z Order the data by the `country` column, then individuals from the same country will be stored in the same files. When you subquently query the data for individuals from a given country, it will execute faster because more data can be skipped. + +Here's how to Z Order a Delta table: + +```python +dt = DeltaTable("tmp") +dt.optimize.z_order([country]) +``` diff --git a/docs/usage/small-file-compaction-with-optimize.md b/docs/usage/optimize/small-file-compaction-with-optimize.md similarity index 100% rename from docs/usage/small-file-compaction-with-optimize.md rename to docs/usage/optimize/small-file-compaction-with-optimize.md diff --git a/mkdocs.yml b/mkdocs.yml index 41f0ee309c..514872e5c8 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -19,12 +19,17 @@ nav: - Usage: - Installation: usage/installation.md - Overview: usage/index.md - - Loading a Delta Table: usage/loading-table.md - - 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 - - Small file compaction: usage/small-file-compaction-with-optimize.md + - Creating a table: usage/create-delta-lake-table.md + - Loading a table: usage/loading-table.md + - Append/overwrite tables: usage/appending-overwriting-delta-lake-table.md + - Examining a table: usage/examining-table.md + - Querying a table: usage/querying-delta-tables.md + - Managing a table: usage/managing-tables.md + - Writing a table: usage/writing-delta-tables.md + - Deleting rows from a table: usage/deleting-rows-from-delta-lake-table.md + - Optimize: + - Small file compaction: usage/optimize/small-file-compaction-with-optimize.md + - Z Order: usage/optimize/delta-lake-z-order.md - API Reference: - api/delta_table.md - api/schema.md From 3173ad7ca05f19b14ebbba081d8365eb848dc47e Mon Sep 17 00:00:00 2001 From: Thomas Frederik Hoeck <44194839+thomasfrederikhoeck@users.noreply.github.com> Date: Wed, 22 Nov 2023 21:37:40 +0100 Subject: [PATCH 26/35] docs: update python docs link in readme.md (#1899) --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9aa98823c6..6121110fb0 100644 --- a/README.md +++ b/README.md @@ -6,7 +6,7 @@

A native Rust library for Delta Lake, with bindings to Python
- Python docs + Python docs · Rust docs · @@ -48,7 +48,7 @@ API that lets you query, inspect, and operate your Delta Lake with ease. [pypi]: https://pypi.org/project/deltalake/ [pypi-dl]: https://img.shields.io/pypi/dm/deltalake?style=flat-square&color=00ADD4 -[py-docs]: https://delta-io.github.io/delta-rs/python/ +[py-docs]: https://delta-io.github.io/delta-rs/ [rs-docs]: https://docs.rs/deltalake/latest/deltalake/ [crates]: https://crates.io/crates/deltalake [crates-dl]: https://img.shields.io/crates/d/deltalake?color=F75101 From fe4fe516bf76271c824642040f185f69e86d45ad Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam <5270024+aersam@users.noreply.github.com> Date: Fri, 24 Nov 2023 19:12:19 +0100 Subject: [PATCH 27/35] fix: use physical name for column name lookup in partitions (#1836) # Description get_actions wrongly assumes that partition_columns from schema and partitionValues from log must be the same. This is not true since partition_columns are logical column names while partitionValues are physical column names. Tests pending # Related Issue(s) - closes #1835 # Documentation https://github.com/delta-io/delta/blob/master/PROTOCOL.md#writer-requirements-for-column-mapping "Track partition values and column level statistics with the physical name of the column in the transaction log." --------- Co-authored-by: Will Jones --- crates/deltalake-core/src/errors.rs | 3 + crates/deltalake-core/src/kernel/error.rs | 3 + crates/deltalake-core/src/kernel/schema.rs | 14 ++++ crates/deltalake-core/src/protocol/mod.rs | 74 ++++++++++++++++++ crates/deltalake-core/src/table/config.rs | 51 ++++++++++++ .../deltalake-core/src/table/state_arrow.rs | 38 ++++++++- ...c-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet | Bin 0 -> 810 bytes ...4-48d9-aa60-438228358f1a.c000.zstd.parquet | Bin 0 -> 890 bytes .../_delta_log/00000000000000000000.crc | 1 + .../_delta_log/00000000000000000000.json | 5 ++ 10 files changed, 186 insertions(+), 3 deletions(-) create mode 100644 crates/deltalake-core/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet create mode 100644 crates/deltalake-core/tests/data/table_with_column_mapping/BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet create mode 100644 crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc create mode 100644 crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json diff --git a/crates/deltalake-core/src/errors.rs b/crates/deltalake-core/src/errors.rs index bd088e9a4f..aaa21a4801 100644 --- a/crates/deltalake-core/src/errors.rs +++ b/crates/deltalake-core/src/errors.rs @@ -211,6 +211,9 @@ pub enum DeltaTableError { #[from] source: crate::kernel::Error, }, + + #[error("Table metadata is invalid: {0}")] + MetadataError(String), } impl From for DeltaTableError { diff --git a/crates/deltalake-core/src/kernel/error.rs b/crates/deltalake-core/src/kernel/error.rs index a37dbdae67..d4110f8f53 100644 --- a/crates/deltalake-core/src/kernel/error.rs +++ b/crates/deltalake-core/src/kernel/error.rs @@ -64,6 +64,9 @@ pub enum Error { /// Invariant expression. line: String, }, + + #[error("Table metadata is invalid: {0}")] + MetadataError(String), } #[cfg(feature = "object_store")] diff --git a/crates/deltalake-core/src/kernel/schema.rs b/crates/deltalake-core/src/kernel/schema.rs index 7694501dca..bc83c05070 100644 --- a/crates/deltalake-core/src/kernel/schema.rs +++ b/crates/deltalake-core/src/kernel/schema.rs @@ -166,6 +166,20 @@ impl StructField { self.nullable } + /// Returns the physical name of the column + /// Equals the name if column mapping is not enabled on table + pub fn physical_name(&self) -> Result<&str, Error> { + // Even on mapping type id the physical name should be there for partitions + let phys_name = self.get_config_value(&ColumnMetadataKey::ColumnMappingPhysicalName); + match phys_name { + None => Ok(&self.name), + Some(MetadataValue::String(s)) => Ok(s), + Some(MetadataValue::Number(_)) => Err(Error::MetadataError( + "Unexpected type for physical name".to_string(), + )), + } + } + #[inline] /// Returns the data type of the column pub const fn data_type(&self) -> &DataType { diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index e2add9b529..264a45eb8b 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -1100,6 +1100,80 @@ mod tests { assert_eq!(expected, actions); } + #[tokio::test] + async fn test_with_column_mapping() { + // test table with column mapping and partitions + let path = "./tests/data/table_with_column_mapping"; + let table = crate::open_table(path).await.unwrap(); + let actions = table.get_state().add_actions_table(true).unwrap(); + let expected_columns: Vec<(&str, ArrayRef)> = vec![ + ( + "path", + Arc::new(array::StringArray::from(vec![ + "BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet", + "8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet", + ])), + ), + ( + "size_bytes", + Arc::new(array::Int64Array::from(vec![890, 810])), + ), + ( + "modification_time", + Arc::new(arrow::array::TimestampMillisecondArray::from(vec![ + 1699946088000, + 1699946088000, + ])), + ), + ( + "data_change", + Arc::new(array::BooleanArray::from(vec![true, true])), + ), + ( + "partition.Company Very Short", + Arc::new(array::StringArray::from(vec!["BMS", "BME"])), + ), + ("num_records", Arc::new(array::Int64Array::from(vec![4, 1]))), + ( + "null_count.Company Very Short", + Arc::new(array::NullArray::new(2)), + ), + ("min.Company Very Short", Arc::new(array::NullArray::new(2))), + ("max.Company Very Short", Arc::new(array::NullArray::new(2))), + ("null_count.Super Name", Arc::new(array::NullArray::new(2))), + ("min.Super Name", Arc::new(array::NullArray::new(2))), + ("max.Super Name", Arc::new(array::NullArray::new(2))), + ( + "tags.INSERTION_TIME", + Arc::new(array::StringArray::from(vec![ + "1699946088000000", + "1699946088000001", + ])), + ), + ( + "tags.MAX_INSERTION_TIME", + Arc::new(array::StringArray::from(vec![ + "1699946088000000", + "1699946088000001", + ])), + ), + ( + "tags.MIN_INSERTION_TIME", + Arc::new(array::StringArray::from(vec![ + "1699946088000000", + "1699946088000001", + ])), + ), + ( + "tags.OPTIMIZE_TARGET_SIZE", + Arc::new(array::StringArray::from(vec!["33554432", "33554432"])), + ), + ]; + let expected = RecordBatch::try_from_iter(expected_columns.clone()).unwrap(); + + assert_eq!(expected, actions); + } + #[tokio::test] async fn test_with_stats() { // test table with stats diff --git a/crates/deltalake-core/src/table/config.rs b/crates/deltalake-core/src/table/config.rs index 5b82b401b6..79130de028 100644 --- a/crates/deltalake-core/src/table/config.rs +++ b/crates/deltalake-core/src/table/config.rs @@ -294,6 +294,14 @@ impl<'a> TableConfig<'a> { .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) .unwrap_or_default() } + + /// Return the column mapping mode according to delta.columnMapping.mode + pub fn column_mapping_mode(&self) -> ColumnMappingMode { + self.0 + .get(DeltaConfigKey::ColumnMappingMode.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) + .unwrap_or_default() + } } #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] @@ -394,6 +402,49 @@ impl FromStr for CheckpointPolicy { } } +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] +/// The Column Mapping modes used for reading and writing data +#[serde(rename_all = "camelCase")] +pub enum ColumnMappingMode { + /// No column mapping is applied + None, + /// Columns are mapped by their field_id in parquet + Id, + /// Columns are mapped to a physical name + Name, +} + +impl Default for ColumnMappingMode { + fn default() -> Self { + Self::None + } +} + +impl AsRef for ColumnMappingMode { + fn as_ref(&self) -> &str { + match self { + Self::None => "none", + Self::Id => "id", + Self::Name => "name", + } + } +} + +impl FromStr for ColumnMappingMode { + type Err = DeltaTableError; + + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "none" => Ok(Self::None), + "id" => Ok(Self::Id), + "name" => Ok(Self::Name), + _ => Err(DeltaTableError::Generic( + "Invalid string for ColumnMappingMode".into(), + )), + } + } +} + const SECONDS_PER_MINUTE: u64 = 60; const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; diff --git a/crates/deltalake-core/src/table/state_arrow.rs b/crates/deltalake-core/src/table/state_arrow.rs index 9d82c87326..3dbb460879 100644 --- a/crates/deltalake-core/src/table/state_arrow.rs +++ b/crates/deltalake-core/src/table/state_arrow.rs @@ -16,6 +16,7 @@ use arrow_array::{ use arrow_schema::{DataType, Field, Fields, TimeUnit}; use itertools::Itertools; +use super::config::ColumnMappingMode; use super::state::DeltaTableState; use crate::errors::DeltaTableError; use crate::kernel::{DataType as DeltaDataType, StructType}; @@ -145,7 +146,7 @@ impl DeltaTableState { flatten: bool, ) -> Result { let metadata = self.current_metadata().ok_or(DeltaTableError::NoMetadata)?; - + let column_mapping_mode = self.table_config().column_mapping_mode(); let partition_column_types: Vec = metadata .partition_columns .iter() @@ -167,13 +168,44 @@ impl DeltaTableState { }) .collect::>(); + let physical_name_to_logical_name = match column_mapping_mode { + ColumnMappingMode::None => HashMap::with_capacity(0), // No column mapping, no need for this HashMap + ColumnMappingMode::Id | ColumnMappingMode::Name => metadata + .partition_columns + .iter() + .map(|name| -> Result<_, DeltaTableError> { + let physical_name = metadata + .schema + .field_with_name(name) + .or(Err(DeltaTableError::MetadataError(format!( + "Invalid partition column {0}", + name + ))))? + .physical_name() + .map_err(|e| DeltaTableError::Kernel { source: e })?; + Ok((physical_name, name.as_str())) + }) + .collect::, DeltaTableError>>()?, + }; // Append values for action in self.files() { for (name, maybe_value) in action.partition_values.iter() { + let logical_name = match column_mapping_mode { + ColumnMappingMode::None => name.as_str(), + ColumnMappingMode::Id | ColumnMappingMode::Name => { + physical_name_to_logical_name.get(name.as_str()).ok_or( + DeltaTableError::MetadataError(format!( + "Invalid partition column {0}", + name + )), + )? + } + }; if let Some(value) = maybe_value { - builders.get_mut(name.as_str()).unwrap().append_value(value); + builders.get_mut(logical_name).unwrap().append_value(value); + // Unwrap is safe here since the name exists in the mapping where we check validity already } else { - builders.get_mut(name.as_str()).unwrap().append_null(); + builders.get_mut(logical_name).unwrap().append_null(); } } } diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet b/crates/deltalake-core/tests/data/table_with_column_mapping/8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f8aadf03ba25640c23fdd93181f5f409688a06de GIT binary patch literal 810 zcmb7@-HOvd6vwAg*IeXw!UO__fh`r1+3hw>H$@gi1qEG-)Ehpa!AVuE9D< zse<0&U@%SNFxZa=y&#OkZU9L%38FAM3}fg{r%`{G^l!Q@a&5zL9k*?|)N=Y{$8@%5 zP|G&#cP;yUt8L!AqJ(Kb;{9K~2!&jeI}-HQt&;;pQktSs2Ao(M&ku)7+Zx7 zV!U}26h-jj&AW&Os~!p>cv4R)x3qZIZ z5gtW^7q3767?uQ{T|@<5(m`@$krLm;u0`Z|qaI*t!r>{Gnx5N~=RDI5Jr@9wd@%M~ zUb@49%Cp^je5?EI>VP^kGW-40+Kp@3L#31G;>)L}u5*un8ta$Vj^4d;=FQ?@=i$e_ z%iRyR=c3hH>+0L$(7mq<`!93T@7}N4$Dem9L*1DO__BO6B5-2rSS<8$aQ<9qn8d*X z2ZY!t2a|?ptC`7Mu2#)!>UchvQMJ65RF1w_mAQK&hu` s3p86S=k>lR$?Zi|Cy))#@iDbnbaKh`sOQSt&n%Cl3Gj*opu~RNA9w2*)c^nh literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc b/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc new file mode 100644 index 0000000000..cb4c652bec --- /dev/null +++ b/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.crc @@ -0,0 +1 @@ +{"txnId":"0e8eece8-347f-4c77-bc4f-daf3a5985dc9","tableSizeBytes":1700,"numFiles":2,"numMetadata":1,"numProtocol":1,"setTransactions":[],"domainMetadata":[],"metadata":{"id":"592de637-dd77-4aaa-af00-97d723a7f1f1","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"Company Very Short\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":1,\"delta.columnMapping.physicalName\":\"col-173b4db9-b5ad-427f-9e75-516aae37fbbb\"}},{\"name\":\"Super Name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":2,\"delta.columnMapping.physicalName\":\"col-3877fd94-0973-4941-ac6b-646849a1ff65\"}}]}","partitionColumns":["Company Very Short"],"configuration":{"delta.columnMapping.mode":"name","delta.autoOptimize.optimizeWrite":"true","delta.columnMapping.maxColumnId":"2","delta.targetFileSize":"33554432","delta.tuneFileSizesForRewrites":"true"},"createdTime":1699946083038},"protocol":{"minReaderVersion":2,"minWriterVersion":5},"histogramOpt":{"sortedBinBoundaries":[0,8192,16384,32768,65536,131072,262144,524288,1048576,2097152,4194304,8388608,12582912,16777216,20971520,25165824,29360128,33554432,37748736,41943040,50331648,58720256,67108864,75497472,83886080,92274688,100663296,109051904,117440512,125829120,130023424,134217728,138412032,142606336,146800640,150994944,167772160,184549376,201326592,218103808,234881024,251658240,268435456,285212672,301989888,318767104,335544320,352321536,369098752,385875968,402653184,419430400,436207616,452984832,469762048,486539264,503316480,520093696,536870912,553648128,570425344,587202560,603979776,671088640,738197504,805306368,872415232,939524096,1006632960,1073741824,1140850688,1207959552,1275068416,1342177280,1409286144,1476395008,1610612736,1744830464,1879048192,2013265920,2147483648,2415919104,2684354560,2952790016,3221225472,3489660928,3758096384,4026531840,4294967296,8589934592,17179869184,34359738368,68719476736,137438953472,274877906944],"fileCounts":[2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"totalBytes":[1700,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]},"allFiles":[{"path":"8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet","partitionValues":{"col-173b4db9-b5ad-427f-9e75-516aae37fbbb":"BME"},"size":810,"modificationTime":1699946088000,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Timothy Lamb\"},\"maxValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Timothy Lamb\"},\"nullCount\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":0}}","tags":{"INSERTION_TIME":"1699946088000001","MIN_INSERTION_TIME":"1699946088000001","MAX_INSERTION_TIME":"1699946088000001","OPTIMIZE_TARGET_SIZE":"33554432"}},{"path":"BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet","partitionValues":{"col-173b4db9-b5ad-427f-9e75-516aae37fbbb":"BMS"},"size":890,"modificationTime":1699946088000,"dataChange":false,"stats":"{\"numRecords\":4,\"minValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Anthony Johnson\"},\"maxValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Stephanie Mcgrath\"},\"nullCount\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":0}}","tags":{"INSERTION_TIME":"1699946088000000","MIN_INSERTION_TIME":"1699946088000000","MAX_INSERTION_TIME":"1699946088000000","OPTIMIZE_TARGET_SIZE":"33554432"}}]} diff --git a/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..c7f52c72d3 --- /dev/null +++ b/crates/deltalake-core/tests/data/table_with_column_mapping/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1699946089972,"userId":"2797914831036774","userName":"censoredmail@bmsuisse.ch","operation":"WRITE","operationParameters":{"mode":"Overwrite","statsOnLoad":false,"partitionBy":"[\"Company Very Short\"]"},"notebook":{"notebookId":"3271485675102593"},"clusterId":"0428-070410-lm8e9giw","isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{"numFiles":"2","numOutputRows":"5","numOutputBytes":"1700"},"tags":{"restoresDeletedRows":"false"},"engineInfo":"Databricks-Runtime/13.3.x-photon-scala2.12","txnId":"0e8eece8-347f-4c77-bc4f-daf3a5985dc9"}} +{"metaData":{"id":"592de637-dd77-4aaa-af00-97d723a7f1f1","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"Company Very Short\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":1,\"delta.columnMapping.physicalName\":\"col-173b4db9-b5ad-427f-9e75-516aae37fbbb\"}},{\"name\":\"Super Name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":2,\"delta.columnMapping.physicalName\":\"col-3877fd94-0973-4941-ac6b-646849a1ff65\"}}]}","partitionColumns":["Company Very Short"],"configuration":{"delta.columnMapping.mode":"name","delta.autoOptimize.optimizeWrite":"true","delta.columnMapping.maxColumnId":"2","delta.targetFileSize":"33554432","delta.tuneFileSizesForRewrites":"true"},"createdTime":1699946083038}} +{"protocol":{"minReaderVersion":2,"minWriterVersion":5}} +{"add":{"path":"BH/part-00000-4d6e745c-8e04-48d9-aa60-438228358f1a.c000.zstd.parquet","partitionValues":{"col-173b4db9-b5ad-427f-9e75-516aae37fbbb":"BMS"},"size":890,"modificationTime":1699946088000,"dataChange":true,"stats":"{\"numRecords\":4,\"minValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Anthony Johnson\"},\"maxValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Stephanie Mcgrath\"},\"nullCount\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":0}}","tags":{"INSERTION_TIME":"1699946088000000","MIN_INSERTION_TIME":"1699946088000000","MAX_INSERTION_TIME":"1699946088000000","OPTIMIZE_TARGET_SIZE":"33554432"}}} +{"add":{"path":"8v/part-00001-69b4a452-aeac-4ffa-bf5c-a0c2833d05eb.c000.zstd.parquet","partitionValues":{"col-173b4db9-b5ad-427f-9e75-516aae37fbbb":"BME"},"size":810,"modificationTime":1699946088000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Timothy Lamb\"},\"maxValues\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":\"Timothy Lamb\"},\"nullCount\":{\"col-3877fd94-0973-4941-ac6b-646849a1ff65\":0}}","tags":{"INSERTION_TIME":"1699946088000001","MIN_INSERTION_TIME":"1699946088000001","MAX_INSERTION_TIME":"1699946088000001","OPTIMIZE_TARGET_SIZE":"33554432"}}} From c38b518b05b2a238d23bdf1c295e1d6bad48b0cc Mon Sep 17 00:00:00 2001 From: Marijn Valk Date: Fri, 24 Nov 2023 19:55:11 +0100 Subject: [PATCH 28/35] ci: run doctest in CI for Python API examples (#1840) --- .github/workflows/python_build.yml | 2 +- python/Makefile | 2 +- python/conftest.py | 20 ++ python/deltalake/table.py | 435 ++++++++++++++++------------- python/pyproject.toml | 1 + 5 files changed, 268 insertions(+), 192 deletions(-) create mode 100644 python/conftest.py diff --git a/.github/workflows/python_build.yml b/.github/workflows/python_build.yml index 223c13f531..51626310be 100644 --- a/.github/workflows/python_build.yml +++ b/.github/workflows/python_build.yml @@ -118,7 +118,7 @@ jobs: - name: Run tests run: | source venv/bin/activate - python -m pytest -m '((s3 or azure) and integration) or not integration and not benchmark' + python -m pytest -m '((s3 or azure) and integration) or not integration and not benchmark' --doctest-modules - name: Test without pandas run: | diff --git a/python/Makefile b/python/Makefile index 5461f70d9e..b876c51a7e 100644 --- a/python/Makefile +++ b/python/Makefile @@ -75,7 +75,7 @@ check-python: ## Run check on Python .PHONY: unit-test unit-test: ## Run unit test $(info --- Run Python unit-test ---) - python -m pytest + python -m pytest --doctest-modules .PHONY: test-pyspark test-pyspark: diff --git a/python/conftest.py b/python/conftest.py new file mode 100644 index 0000000000..3ec2df8249 --- /dev/null +++ b/python/conftest.py @@ -0,0 +1,20 @@ +import os +import pathlib + +import pytest + + +@pytest.fixture(autouse=True) +def doctest_setup( + request: pytest.FixtureRequest, + monkeypatch: pytest.MonkeyPatch, + tmp_path: pathlib.Path, +): + if isinstance(request.node, pytest.DoctestItem): + # disable color for doctests so we don't have to include + # escape codes in docstrings + monkeypatch.setitem(os.environ, "NO_COLOR", "1") + # Explicitly set the column width + monkeypatch.setitem(os.environ, "COLUMNS", "80") + # Work in a temporary directory + monkeypatch.chdir(str(tmp_path)) diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 4ef07e5676..941e0d1fce 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -32,13 +32,13 @@ if TYPE_CHECKING: import pandas -from ._internal import DeltaDataChecker as _DeltaDataChecker -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 deltalake._internal import DeltaDataChecker as _DeltaDataChecker +from deltalake._internal import RawDeltaTable +from deltalake._util import encode_partition_value +from deltalake.data_catalog import DataCatalog +from deltalake.exceptions import DeltaProtocolError +from deltalake.fs import DeltaStorageHandler +from deltalake.schema import Schema MAX_SUPPORTED_READER_VERSION = 1 MAX_SUPPORTED_WRITER_VERSION = 2 @@ -521,30 +521,32 @@ def update( Examples: - Update some row values with SQL predicate. This is equivalent to `UPDATE table SET deleted = true WHERE id = '5'` + Update some row values with SQL predicate. This is equivalent to `UPDATE table SET deleted = true WHERE id = '3'` - ``` - from deltalake import DeltaTable - dt = DeltaTable("tmp") - dt.update(predicate="id = '5'", updates = {"deleted": 'True'}) - ``` + >>> from deltalake import write_deltalake, DeltaTable + >>> import pandas as pd + >>> df = pd.DataFrame({"id": ["1", "2", "3"], "deleted": [False, False, False], "price": [10., 15., 20.]}) + >>> write_deltalake("tmp", df) + >>> dt = DeltaTable("tmp") + >>> dt.update(predicate="id = '3'", updates = {"deleted": 'True'}) + + {'num_added_files': 1, 'num_removed_files': 1, 'num_updated_rows': 1, 'num_copied_rows': 2, 'execution_time_ms': ..., 'scan_time_ms': ...} - Update all row values. This is equivalent to - ``UPDATE table SET deleted = true, id = concat(id, '_old')``. - ``` - from deltalake import DeltaTable - dt = DeltaTable("tmp") - dt.update(updates = {"deleted": 'True', "id": "concat(id, '_old')"}) - ``` + Update all row values. This is equivalent to + ``UPDATE table SET deleted = true, id = concat(id, '_old')``. + + >>> dt.update(updates = {"deleted": 'True', "id": "concat(id, '_old')"}) + + + {'num_added_files': 1, 'num_removed_files': 1, 'num_updated_rows': 3, 'num_copied_rows': 0, 'execution_time_ms': ..., 'scan_time_ms': ...} + + To use Python objects instead of SQL strings, use the `new_values` parameter + instead of the `updates` parameter. For example, this is equivalent to + ``UPDATE table SET price = 150.10 WHERE id = '1'`` + + >>> dt.update(predicate="id = '1_old'", new_values = {"price": 150.10}) + {'num_added_files': 1, 'num_removed_files': 1, 'num_updated_rows': 1, 'num_copied_rows': 2, 'execution_time_ms': ..., 'scan_time_ms': ...} - To use Python objects instead of SQL strings, use the `new_values` parameter - instead of the `updates` parameter. For example, this is equivalent to - ``UPDATE table SET price = 150.10 WHERE id = '5'`` - ``` - from deltalake import DeltaTable - dt = DeltaTable("tmp") - dt.update(predicate="id = '5'", new_values = {"price": 150.10}) - ``` """ if updates is None and new_values is not None: updates = {} @@ -838,30 +840,26 @@ def get_add_actions(self, flatten: bool = False) -> pyarrow.RecordBatch: a PyArrow RecordBatch containing the add action data. Example: - ``` - from deltalake import DeltaTable, write_deltalake - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - write_deltalake("tmp", data, partition_by=["x"]) - dt = DeltaTable("tmp") - dt.get_add_actions().to_pandas() - ``` - ``` - path size_bytes modification_time data_change partition_values num_records null_count min max - 0 x=2/0-91820cbf-f698-45fb-886d-5d5f5669530b-0.p... 565 1970-01-20 08:40:08.071 True {'x': 2} 1 {'y': 0} {'y': 5} {'y': 5} - 1 x=3/0-91820cbf-f698-45fb-886d-5d5f5669530b-0.p... 565 1970-01-20 08:40:08.071 True {'x': 3} 1 {'y': 0} {'y': 6} {'y': 6} - 2 x=1/0-91820cbf-f698-45fb-886d-5d5f5669530b-0.p... 565 1970-01-20 08:40:08.071 True {'x': 1} 1 {'y': 0} {'y': 4} {'y': 4} - ``` - ``` - dt.get_add_actions(flatten=True).to_pandas() - ``` - ``` - path size_bytes modification_time data_change partition.x num_records null_count.y min.y max.y - 0 x=2/0-91820cbf-f698-45fb-886d-5d5f5669530b-0.p... 565 1970-01-20 08:40:08.071 True 2 1 0 5 5 - 1 x=3/0-91820cbf-f698-45fb-886d-5d5f5669530b-0.p... 565 1970-01-20 08:40:08.071 True 3 1 0 6 6 - 2 x=1/0-91820cbf-f698-45fb-886d-5d5f5669530b-0.p... 565 1970-01-20 08:40:08.071 True 1 1 0 4 4 - ``` + >>> from pprint import pprint + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data, partition_by=["x"]) + >>> dt = DeltaTable("tmp") + >>> df = dt.get_add_actions().to_pandas() + >>> df["path"].sort_values(ignore_index=True) + 0 x=1/0-... + 1 x=2/0-... + 2 x=3/0-... + ... + >>> df = dt.get_add_actions(flatten=True).to_pandas() + >>> df["partition.x"].sort_values(ignore_index=True) + 0 1 + 1 2 + 2 3 + ... + """ return self._table.get_add_actions(flatten) @@ -986,21 +984,29 @@ def when_matched_update( TableMerger: TableMerger Object Examples: - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - ( \ - dt.merge( \ - source=data, \ - predicate="target.x = source.x", \ - source_alias="source", \ - target_alias="target") \ - .when_matched_update(updates={"x": "source.x", "y": "source.y"}) \ - .execute() \ - ) - ``` + + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data) + >>> dt = DeltaTable("tmp") + >>> new_data = pa.table({"x": [1], "y": [7]}) + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate="target.x = source.x", + ... source_alias="source", + ... target_alias="target") + ... .when_matched_update(updates={"x": "source.x", "y": "source.y"}) + ... .execute() + ... ) + {'num_source_rows': 1, 'num_target_rows_inserted': 0, 'num_target_rows_updated': 1, 'num_target_rows_deleted': 0, 'num_target_rows_copied': 2, 'num_output_rows': 3, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas() + x y + 0 1 7 + 1 2 5 + 2 3 6 + """ if isinstance(self.matched_update_updates, list) and isinstance( self.matched_update_predicate, list @@ -1013,7 +1019,7 @@ def when_matched_update( return self def when_matched_update_all(self, predicate: Optional[str] = None) -> "TableMerger": - """Updating all source fields to target fields, source and target are required to have the same field names. + """Updating all source fields to target fields, source and target are required to have the same field names. If a ``predicate`` is specified, then it must evaluate to true for the row to be updated. Args: @@ -1021,24 +1027,31 @@ def when_matched_update_all(self, predicate: Optional[str] = None) -> "TableMerg Returns: TableMerger: TableMerger Object - + Examples: - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - (\ - dt.merge( \ - source=data, \ - predicate='target.x = source.x', \ - source_alias='source', \ - target_alias='target') \ - .when_matched_update_all() \ - .execute() \ - ) - ``` + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data) + >>> dt = DeltaTable("tmp") + >>> new_data = pa.table({"x": [1], "y": [7]}) + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate="target.x = source.x", + ... source_alias="source", + ... target_alias="target") + ... .when_matched_update_all() + ... .execute() + ... ) + {'num_source_rows': 1, 'num_target_rows_inserted': 0, 'num_target_rows_updated': 1, 'num_target_rows_deleted': 0, 'num_target_rows_copied': 2, 'num_output_rows': 3, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas() + x y + 0 1 7 + 1 2 5 + 2 3 6 + """ src_alias = (self.source_alias + ".") if self.source_alias is not None else "" @@ -1069,40 +1082,50 @@ def when_matched_delete(self, predicate: Optional[str] = None) -> "TableMerger": Returns: TableMerger: TableMerger Object - + Examples: - Delete on a predicate - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - ( \ - dt.merge( \ - source=data, \ - predicate='target.x = source.x', \ - source_alias='source', \ - target_alias='target') \ - .when_matched_delete( \ - predicate = "source.deleted = true") \ - .execute() \ - ``` - Delete all records that were matched - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - ( \ - dt.merge( \ - source=data, \ - predicate='target.x = source.x', \ - source_alias='source', \ - target_alias='target') \ - .when_matched_delete() \ - .execute() \ - ``` + Delete on a predicate + + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data) + >>> dt = DeltaTable("tmp") + >>> new_data = pa.table({"x": [2, 3], "deleted": [False, True]}) + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate='target.x = source.x', + ... source_alias='source', + ... target_alias='target') + ... .when_matched_delete( + ... predicate="source.deleted = true") + ... .execute() + ... ) + {'num_source_rows': 2, 'num_target_rows_inserted': 0, 'num_target_rows_updated': 0, 'num_target_rows_deleted': 1, 'num_target_rows_copied': 2, 'num_output_rows': 2, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas().sort_values("x", ignore_index=True) + x y + 0 1 4 + 1 2 5 + + Delete all records that were matched + + >>> dt = DeltaTable("tmp") + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate='target.x = source.x', + ... source_alias='source', + ... target_alias='target') + ... .when_matched_delete() + ... .execute() + ... ) + {'num_source_rows': 2, 'num_target_rows_inserted': 0, 'num_target_rows_updated': 0, 'num_target_rows_deleted': 1, 'num_target_rows_copied': 1, 'num_output_rows': 1, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas() + x y + 0 1 4 + """ if self.matched_delete_all is not None: raise ValueError( @@ -1131,28 +1154,36 @@ def when_not_matched_insert( Returns: TableMerger: TableMerger Object - + Examples: - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - ( \ - dt.merge( \ - source=data, \ - predicate='target.x = source.x', \ - source_alias='source', \ - target_alias='target') \ - .when_not_matched_insert( \ - updates = { \ - "x": "source.x", \ - "y": "source.y", \ - }) \ - .execute() \ - ) - ``` + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data) + >>> dt = DeltaTable("tmp") + >>> new_data = pa.table({"x": [4], "y": [7]}) + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate='target.x = source.x', + ... source_alias='source', + ... target_alias='target') + ... .when_not_matched_insert( + ... updates = { + ... "x": "source.x", + ... "y": "source.y", + ... }) + ... .execute() + ... ) + {'num_source_rows': 1, 'num_target_rows_inserted': 1, 'num_target_rows_updated': 0, 'num_target_rows_deleted': 0, 'num_target_rows_copied': 3, 'num_output_rows': 4, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas().sort_values("x", ignore_index=True) + x y + 0 1 4 + 1 2 5 + 2 3 6 + 3 4 7 + """ if isinstance(self.not_matched_insert_updates, list) and isinstance( @@ -1169,8 +1200,8 @@ def when_not_matched_insert( def when_not_matched_insert_all( self, predicate: Optional[str] = None ) -> "TableMerger": - """Insert a new row to the target table, updating all source fields to target fields. Source and target are - required to have the same field names. If a ``predicate`` is specified, then it must evaluate to true for + """Insert a new row to the target table, updating all source fields to target fields. Source and target are + required to have the same field names. If a ``predicate`` is specified, then it must evaluate to true for the new row to be inserted. Args: @@ -1178,25 +1209,32 @@ def when_not_matched_insert_all( Returns: TableMerger: TableMerger Object - + Examples: - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - ( \ - dt \ - .merge( \ - source=data, \ - predicate='target.x = source.x', \ - source_alias='source', \ - target_alias='target') \ - .when_not_matched_insert_all() \ - .execute() \ - ) - ``` + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data) + >>> dt = DeltaTable("tmp") + >>> new_data = pa.table({"x": [4], "y": [7]}) + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate='target.x = source.x', + ... source_alias='source', + ... target_alias='target') + ... .when_not_matched_insert_all() + ... .execute() + ... ) + {'num_source_rows': 1, 'num_target_rows_inserted': 1, 'num_target_rows_updated': 0, 'num_target_rows_deleted': 0, 'num_target_rows_copied': 3, 'num_output_rows': 4, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas().sort_values("x", ignore_index=True) + x y + 0 1 4 + 1 2 5 + 2 3 6 + 3 4 7 + """ src_alias = (self.source_alias + ".") if self.source_alias is not None else "" @@ -1228,24 +1266,33 @@ def when_not_matched_by_source_update( Returns: TableMerger: TableMerger Object - - ``` - from deltalake import DeltaTable - import pyarrow as pa - data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) - dt = DeltaTable("tmp") - ( \ - dt.merge( \ - source=data, \ - predicate='target.x = source.x', \ - source_alias='source', \ - target_alias='target') \ - .when_not_matched_by_source_update( \ - predicate = "y > 3", \ - updates = {"y": "0"}) \ - .execute() \ - ) \ - ``` + + Examples: + + >>> from deltalake import DeltaTable, write_deltalake + >>> import pyarrow as pa + >>> data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> write_deltalake("tmp", data) + >>> dt = DeltaTable("tmp") + >>> new_data = pa.table({"x": [2, 3, 4]}) + >>> ( + ... dt.merge( + ... source=new_data, + ... predicate='target.x = source.x', + ... source_alias='source', + ... target_alias='target') + ... .when_not_matched_by_source_update( + ... predicate = "y > 3", + ... updates = {"y": "0"}) + ... .execute() + ... ) + {'num_source_rows': 3, 'num_target_rows_inserted': 0, 'num_target_rows_updated': 1, 'num_target_rows_deleted': 0, 'num_target_rows_copied': 2, 'num_output_rows': 3, 'num_target_files_added': 1, 'num_target_files_removed': 1, 'execution_time_ms': ..., 'scan_time_ms': ..., 'rewrite_time_ms': ...} + >>> dt.to_pandas().sort_values("x", ignore_index=True) + x y + 0 1 0 + 1 2 5 + 2 3 6 + """ if isinstance(self.not_matched_by_source_update_updates, list) and isinstance( @@ -1369,15 +1416,19 @@ def compact( Returns: the metrics from optimize - Example: - ``` - # 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) - ``` + Examples: + Use a timedelta object to specify the seconds, minutes or hours of the interval. + + >>> from deltalake import DeltaTable, write_deltalake + >>> from datetime import timedelta + >>> import pyarrow as pa + >>> write_deltalake("tmp", pa.table({"x": [1], "y": [4]})) + >>> write_deltalake("tmp", pa.table({"x": [2], "y": [5]}), mode="append") + >>> dt = DeltaTable("tmp") + >>> time_delta = timedelta(minutes=10) + >>> dt.optimize.compact(min_commit_interval=time_delta) + {'numFilesAdded': 1, 'numFilesRemoved': 2, 'filesAdded': ..., 'filesRemoved': ..., 'partitionsOptimized': 1, 'numBatches': 2, 'totalConsideredFiles': 2, 'totalFilesSkipped': 0, 'preserveInsertionOrder': True} + """ if isinstance(min_commit_interval, timedelta): min_commit_interval = int(min_commit_interval.total_seconds()) @@ -1419,15 +1470,19 @@ def z_order( Returns: the metrics from optimize - Example: - ``` - # 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) - ``` + Examples: + Use a timedelta object to specify the seconds, minutes or hours of the interval. + + >>> from deltalake import DeltaTable, write_deltalake + >>> from datetime import timedelta + >>> import pyarrow as pa + >>> write_deltalake("tmp", pa.table({"x": [1], "y": [4]})) + >>> write_deltalake("tmp", pa.table({"x": [2], "y": [5]}), mode="append") + >>> dt = DeltaTable("tmp") + >>> time_delta = timedelta(minutes=10) + >>> dt.optimize.z_order(["x"], min_commit_interval=time_delta) + {'numFilesAdded': 1, 'numFilesRemoved': 2, 'filesAdded': ..., 'filesRemoved': ..., 'partitionsOptimized': 0, 'numBatches': 1, 'totalConsideredFiles': 2, 'totalFilesSkipped': 0, 'preserveInsertionOrder': True} + """ if isinstance(min_commit_interval, timedelta): min_commit_interval = int(min_commit_interval.total_seconds()) diff --git a/python/pyproject.toml b/python/pyproject.toml index 09a7e4b37c..aaeda6bfd2 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -93,6 +93,7 @@ known-first-party = ["deltalake"] addopts = "--cov=deltalake -v -m 'not integration and not benchmark'" testpaths = [ "tests", + "deltalake", ] markers = [ "integration: marks tests as integration tests (deselect with '-m \"not integration\"')", From 3ed7df04f1c781cf289f2bed7965c433083da27a Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Fri, 24 Nov 2023 21:23:15 +0100 Subject: [PATCH 29/35] feat(python): add pyarrow to delta compatible schema conversion in writer/merge (#1820) This ports some functionality that @stinodego and I had worked on in Polars. Where we converted a pyarrow schema to a compatible delta schema. It converts the following: - uint -> int - timestamp(any timeunit) -> timestamp(us) I adjusted the functionality to do schema conversion from large to normal when necessary, which is still needed in MERGE as workaround https://github.com/delta-io/delta-rs/issues/1753. Additional things I've added: - Schema conversion for every input in write_deltalake/merge - Add Pandas dataframe conversion - Add Pandas dataframe as input in merge - closes https://github.com/delta-io/delta-rs/issues/686 - closes https://github.com/delta-io/delta-rs/issues/1467 --------- Co-authored-by: Will Jones --- python/deltalake/schema.py | 135 +++++++--- python/deltalake/table.py | 34 ++- python/deltalake/writer.py | 63 +++-- python/licenses/README.md | 8 + .../deltalake_license.txt} | 0 python/licenses/polars_license.txt | 19 ++ python/pyproject.toml | 2 +- python/stubs/pyarrow/__init__.pyi | 13 + python/tests/test_schema.py | 248 +++++++++++++++++- python/tests/test_writer.py | 30 ++- 10 files changed, 483 insertions(+), 69 deletions(-) create mode 100644 python/licenses/README.md rename python/{LICENSE.txt => licenses/deltalake_license.txt} (100%) create mode 100644 python/licenses/polars_license.txt diff --git a/python/deltalake/schema.py b/python/deltalake/schema.py index e6854c3779..a22725fdc5 100644 --- a/python/deltalake/schema.py +++ b/python/deltalake/schema.py @@ -1,9 +1,7 @@ -from typing import TYPE_CHECKING, Tuple, Union +from typing import Generator, Union import pyarrow as pa - -if TYPE_CHECKING: - import pandas as pd +import pyarrow.dataset as ds from ._internal import ArrayType as ArrayType from ._internal import Field as Field @@ -17,34 +15,109 @@ DataType = Union["PrimitiveType", "MapType", "StructType", "ArrayType"] -def delta_arrow_schema_from_pandas( - data: "pd.DataFrame", -) -> Tuple[pa.Table, pa.Schema]: - """ - Infers the schema for the delta table from the Pandas DataFrame. - Necessary because of issues such as: https://github.com/delta-io/delta-rs/issues/686 - - Args: - data: Data to write. +### Inspired from Pola-rs repo - licensed with MIT License, see license in python/licenses/polars_license.txt.### +def _convert_pa_schema_to_delta( + schema: pa.schema, large_dtypes: bool = False +) -> pa.schema: + """Convert a PyArrow schema to a schema compatible with Delta Lake. Converts unsigned to signed equivalent, and + converts all timestamps to `us` timestamps. With the boolean flag large_dtypes you can control if the schema + should keep cast normal to large types in the schema, or from large to normal. - Returns: - A PyArrow Table and the inferred schema for the Delta Table + Args + schema: Source schema + large_dtypes: If True, the pyarrow schema is casted to large_dtypes """ + dtype_map = { + pa.uint8(): pa.int8(), + pa.uint16(): pa.int16(), + pa.uint32(): pa.int32(), + pa.uint64(): pa.int64(), + } + if large_dtypes: + dtype_map = { + **dtype_map, + **{pa.string(): pa.large_string(), pa.binary(): pa.large_binary()}, + } + else: + dtype_map = { + **dtype_map, + **{pa.large_string(): pa.string(), pa.large_binary(): pa.binary()}, + } - table = pa.Table.from_pandas(data) - schema = table.schema - schema_out = [] - for field in schema: - if isinstance(field.type, pa.TimestampType): - f = pa.field( - name=field.name, - type=pa.timestamp("us"), - nullable=field.nullable, - metadata=field.metadata, - ) - schema_out.append(f) + def dtype_to_delta_dtype(dtype: pa.DataType) -> pa.DataType: + # Handle nested types + if isinstance(dtype, (pa.LargeListType, pa.ListType)): + return list_to_delta_dtype(dtype) + elif isinstance(dtype, pa.StructType): + return struct_to_delta_dtype(dtype) + elif isinstance(dtype, pa.TimestampType): + return pa.timestamp( + "us" + ) # TODO(ion): propagate also timezone information during writeonce we can properly read TZ in delta schema + try: + return dtype_map[dtype] + except KeyError: + return dtype + + def list_to_delta_dtype( + dtype: Union[pa.LargeListType, pa.ListType], + ) -> Union[pa.LargeListType, pa.ListType]: + nested_dtype = dtype.value_type + nested_dtype_cast = dtype_to_delta_dtype(nested_dtype) + if large_dtypes: + return pa.large_list(nested_dtype_cast) else: - schema_out.append(field) - schema = pa.schema(schema_out, metadata=schema.metadata) - table = table.cast(target_schema=schema) - return table, schema + return pa.list_(nested_dtype_cast) + + def struct_to_delta_dtype(dtype: pa.StructType) -> pa.StructType: + fields = [dtype[i] for i in range(dtype.num_fields)] + fields_cast = [f.with_type(dtype_to_delta_dtype(f.type)) for f in fields] + return pa.struct(fields_cast) + + return pa.schema([f.with_type(dtype_to_delta_dtype(f.type)) for f in schema]) + + +def _cast_schema_to_recordbatchreader( + reader: pa.RecordBatchReader, schema: pa.schema +) -> Generator[pa.RecordBatch, None, None]: + """Creates recordbatch generator.""" + for batch in reader: + yield pa.Table.from_batches([batch]).cast(schema).to_batches()[0] + + +def convert_pyarrow_recordbatchreader( + data: pa.RecordBatchReader, large_dtypes: bool +) -> pa.RecordBatchReader: + """Converts a PyArrow RecordBatchReader to a PyArrow RecordBatchReader with a compatible delta schema""" + schema = _convert_pa_schema_to_delta(data.schema, large_dtypes=large_dtypes) + + data = pa.RecordBatchReader.from_batches( + schema, + _cast_schema_to_recordbatchreader(data, schema), + ) + return data + + +def convert_pyarrow_recordbatch( + data: pa.RecordBatch, large_dtypes: bool +) -> pa.RecordBatchReader: + """Converts a PyArrow RecordBatch to a PyArrow RecordBatchReader with a compatible delta schema""" + schema = _convert_pa_schema_to_delta(data.schema, large_dtypes=large_dtypes) + data = pa.Table.from_batches([data]).cast(schema).to_reader() + return data + + +def convert_pyarrow_table(data: pa.Table, large_dtypes: bool) -> pa.RecordBatchReader: + """Converts a PyArrow table to a PyArrow RecordBatchReader with a compatible delta schema""" + schema = _convert_pa_schema_to_delta(data.schema, large_dtypes=large_dtypes) + data = data.cast(schema).to_reader() + return data + + +def convert_pyarrow_dataset( + data: ds.Dataset, large_dtypes: bool +) -> pa.RecordBatchReader: + """Converts a PyArrow dataset to a PyArrow RecordBatchReader with a compatible delta schema""" + data = data.scanner().to_reader() + data = convert_pyarrow_recordbatchreader(data, large_dtypes) + return data diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 941e0d1fce..b238af7929 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -20,6 +20,7 @@ ) import pyarrow +import pyarrow.dataset as ds import pyarrow.fs as pa_fs from pyarrow.dataset import ( Expression, @@ -596,7 +597,13 @@ def optimize( def merge( self, - source: Union[pyarrow.Table, pyarrow.RecordBatch, pyarrow.RecordBatchReader], + source: Union[ + pyarrow.Table, + pyarrow.RecordBatch, + pyarrow.RecordBatchReader, + ds.Dataset, + "pandas.DataFrame", + ], predicate: str, source_alias: Optional[str] = None, target_alias: Optional[str] = None, @@ -619,17 +626,28 @@ def merge( invariants = self.schema().invariants checker = _DeltaDataChecker(invariants) + from .schema import ( + convert_pyarrow_dataset, + convert_pyarrow_recordbatch, + convert_pyarrow_recordbatchreader, + convert_pyarrow_table, + ) + if isinstance(source, pyarrow.RecordBatchReader): - schema = source.schema + source = convert_pyarrow_recordbatchreader(source, large_dtypes=True) elif isinstance(source, pyarrow.RecordBatch): - schema = source.schema - source = [source] + source = convert_pyarrow_recordbatch(source, large_dtypes=True) elif isinstance(source, pyarrow.Table): - schema = source.schema - source = source.to_reader() + source = convert_pyarrow_table(source, large_dtypes=True) + elif isinstance(source, ds.Dataset): + source = convert_pyarrow_dataset(source, large_dtypes=True) + elif isinstance(source, pandas.DataFrame): + source = convert_pyarrow_table( + pyarrow.Table.from_pandas(source), large_dtypes=True + ) else: raise TypeError( - f"{type(source).__name__} is not a valid input. Only PyArrow RecordBatchReader, RecordBatch or Table are valid inputs for source." + f"{type(source).__name__} is not a valid input. Only PyArrow RecordBatchReader, RecordBatch, Table or Pandas DataFrame are valid inputs for source." ) def validate_batch(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: @@ -637,7 +655,7 @@ def validate_batch(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: return batch source = pyarrow.RecordBatchReader.from_batches( - schema, (validate_batch(batch) for batch in source) + source.schema, (validate_batch(batch) for batch in source) ) return TableMerger( diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 358f7fed41..7d6719985d 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -34,14 +34,18 @@ import pyarrow.fs as pa_fs from pyarrow.lib import RecordBatchReader -from deltalake.schema import delta_arrow_schema_from_pandas - from ._internal import DeltaDataChecker as _DeltaDataChecker from ._internal import batch_distinct from ._internal import convert_to_deltalake as _convert_to_deltalake from ._internal import write_new_deltalake as write_deltalake_pyarrow from ._internal import write_to_deltalake as write_deltalake_rust from .exceptions import DeltaProtocolError, TableNotFoundError +from .schema import ( + convert_pyarrow_dataset, + convert_pyarrow_recordbatch, + convert_pyarrow_recordbatchreader, + convert_pyarrow_table, +) from .table import MAX_SUPPORTED_WRITER_VERSION, DeltaTable try: @@ -161,7 +165,7 @@ def write_deltalake( overwrite_schema: If True, allows updating the schema of the table. storage_options: options passed to the native delta filesystem. Unused if 'filesystem' is defined. partition_filters: the partition filters that will be used for partition overwrite. Only used in pyarrow engine. - large_dtypes: If True, the table schema is checked against large_dtypes + large_dtypes: If True, the data schema is kept in large_dtypes, has no effect on pandas dataframe input """ table, table_uri = try_get_table_and_table_uri(table_or_uri, storage_options) if table is not None: @@ -230,13 +234,35 @@ def write_deltalake( else: data, schema = delta_arrow_schema_from_pandas(data) + table, table_uri = try_get_table_and_table_uri(table_or_uri, storage_options) + + # We need to write against the latest table version + if table: + table.update_incremental() + + if isinstance(data, RecordBatchReader): + data = convert_pyarrow_recordbatchreader(data, large_dtypes) + elif isinstance(data, pa.RecordBatch): + data = convert_pyarrow_recordbatch(data, large_dtypes) + elif isinstance(data, pa.Table): + data = convert_pyarrow_table(data, large_dtypes) + elif isinstance(data, ds.Dataset): + data = convert_pyarrow_dataset(data, large_dtypes) + elif _has_pandas and isinstance(data, pd.DataFrame): + if schema is not None: + data = pa.Table.from_pandas(data, schema=schema) + else: + data = convert_pyarrow_table(pa.Table.from_pandas(data), False) + elif isinstance(data, Iterable): if schema is None: - if isinstance(data, RecordBatchReader): - schema = data.schema - elif isinstance(data, Iterable): - raise ValueError("You must provide schema if data is Iterable") - else: - schema = data.schema + raise ValueError("You must provide schema if data is Iterable") + else: + raise TypeError( + f"{type(data).__name__} is not a valid input. Only PyArrow RecordBatchReader, RecordBatch, Iterable[RecordBatch], Table, Dataset or Pandas DataFrame are valid inputs for source." + ) + + if schema is None: + schema = data.schema if filesystem is not None: raise NotImplementedError( @@ -269,7 +295,7 @@ def write_deltalake( current_version = -1 dtype_map = { - pa.large_string(): pa.string(), # type: ignore + pa.large_string(): pa.string(), } def _large_to_normal_dtype(dtype: pa.DataType) -> pa.DataType: @@ -373,20 +399,9 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: return batch - if isinstance(data, RecordBatchReader): - batch_iter = data - elif isinstance(data, pa.RecordBatch): - batch_iter = [data] - elif isinstance(data, pa.Table): - batch_iter = data.to_batches() - elif isinstance(data, ds.Dataset): - batch_iter = data.to_batches() - else: - batch_iter = data - - data = RecordBatchReader.from_batches( - schema, (validate_batch(batch) for batch in batch_iter) - ) + data = RecordBatchReader.from_batches( + schema, (validate_batch(batch) for batch in data) + ) if file_options is not None: file_options.update(use_compliant_nested_type=False) diff --git a/python/licenses/README.md b/python/licenses/README.md new file mode 100644 index 0000000000..7f8f61c9d4 --- /dev/null +++ b/python/licenses/README.md @@ -0,0 +1,8 @@ +# Licenses +Below are described which licenses apply to the deltalake package and to which areas of the source code. + +### deltalake_license.txt (APACHE 2.0 License) +Applies to the full deltalake package source code. + +### polars_license.txt (MIT License) +Applies solely to the `_convert_pa_schema_to_delta` function in `deltalake/schema.py`. \ No newline at end of file diff --git a/python/LICENSE.txt b/python/licenses/deltalake_license.txt similarity index 100% rename from python/LICENSE.txt rename to python/licenses/deltalake_license.txt diff --git a/python/licenses/polars_license.txt b/python/licenses/polars_license.txt new file mode 100644 index 0000000000..06d01f6abf --- /dev/null +++ b/python/licenses/polars_license.txt @@ -0,0 +1,19 @@ +Copyright (c) 2020 Ritchie Vink + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/python/pyproject.toml b/python/pyproject.toml index aaeda6bfd2..6ffe4ca14c 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -6,7 +6,7 @@ build-backend = "maturin" name = "deltalake" description = "Native Delta Lake Python binding based on delta-rs with Pandas integration" readme = "README.md" -license = {file = "LICENSE.txt"} +license = {file = "licenses/deltalake_license.txt"} requires-python = ">=3.8" keywords = ["deltalake", "delta", "datalake", "pandas", "arrow"] classifiers = [ diff --git a/python/stubs/pyarrow/__init__.pyi b/python/stubs/pyarrow/__init__.pyi index f8c9d152aa..10edfcf663 100644 --- a/python/stubs/pyarrow/__init__.pyi +++ b/python/stubs/pyarrow/__init__.pyi @@ -19,10 +19,23 @@ type_for_alias: Any date32: Any date64: Any decimal128: Any +int8: Any +int16: Any int32: Any +int64: Any +uint8: Any +uint16: Any +uint32: Any +uint64: Any float16: Any float32: Any float64: Any +large_string: Any +string: Any +large_binary: Any +binary: Any +large_list: Any +LargeListType: Any dictionary: Any timestamp: Any TimestampType: Any diff --git a/python/tests/test_schema.py b/python/tests/test_schema.py index f63df0e9fb..6a30ca684e 100644 --- a/python/tests/test_schema.py +++ b/python/tests/test_schema.py @@ -1,10 +1,17 @@ import json -import pyarrow +import pyarrow as pa import pytest from deltalake import DeltaTable, Field -from deltalake.schema import ArrayType, MapType, PrimitiveType, Schema, StructType +from deltalake.schema import ( + ArrayType, + MapType, + PrimitiveType, + Schema, + StructType, + _convert_pa_schema_to_delta, +) def test_table_schema(): @@ -36,7 +43,7 @@ def test_table_schema_pyarrow_simple(): field = schema.field(0) assert len(schema.types) == 1 assert field.name == "id" - assert field.type == pyarrow.int64() + assert field.type == pa.int64() assert field.nullable is True assert field.metadata is None @@ -48,7 +55,7 @@ def test_table_schema_pyarrow_020(): field = schema.field(0) assert len(schema.types) == 1 assert field.name == "value" - assert field.type == pyarrow.int32() + assert field.type == pa.int32() assert field.nullable is True assert field.metadata is None @@ -213,3 +220,236 @@ def test_delta_schema(): schema_without_metadata = schema = Schema(fields) pa_schema = schema_without_metadata.to_pyarrow() assert schema_without_metadata == Schema.from_pyarrow(pa_schema) + + +@pytest.mark.parametrize( + "schema,expected_schema,large_dtypes", + [ + ( + pa.schema([("some_int", pa.uint32()), ("some_string", pa.string())]), + pa.schema([("some_int", pa.int32()), ("some_string", pa.string())]), + False, + ), + ( + pa.schema( + [ + pa.field("some_int", pa.uint32(), nullable=True), + pa.field("some_string", pa.string(), nullable=False), + ] + ), + pa.schema( + [ + pa.field("some_int", pa.int32(), nullable=True), + pa.field("some_string", pa.string(), nullable=False), + ] + ), + False, + ), + ( + pa.schema( + [ + pa.field("some_int", pa.uint32(), nullable=True), + pa.field("some_string", pa.string(), nullable=False), + ] + ), + pa.schema( + [ + pa.field("some_int", pa.int32(), nullable=True), + pa.field("some_string", pa.large_string(), nullable=False), + ] + ), + True, + ), + ( + pa.schema([("some_int", pa.uint32()), ("some_string", pa.string())]), + pa.schema([("some_int", pa.int32()), ("some_string", pa.large_string())]), + True, + ), + ( + pa.schema([("some_int", pa.uint32()), ("some_string", pa.large_string())]), + pa.schema([("some_int", pa.int32()), ("some_string", pa.string())]), + False, + ), + ( + pa.schema( + [ + ("some_int", pa.uint8()), + ("some_int1", pa.uint16()), + ("some_int2", pa.uint32()), + ("some_int3", pa.uint64()), + ] + ), + pa.schema( + [ + ("some_int", pa.int8()), + ("some_int1", pa.int16()), + ("some_int2", pa.int32()), + ("some_int3", pa.int64()), + ] + ), + True, + ), + ( + pa.schema( + [ + ("some_list", pa.list_(pa.string())), + ("some_list_binary", pa.list_(pa.binary())), + ("some_string", pa.large_string()), + ] + ), + pa.schema( + [ + ("some_list", pa.large_list(pa.large_string())), + ("some_list_binary", pa.large_list(pa.large_binary())), + ("some_string", pa.large_string()), + ] + ), + True, + ), + ( + pa.schema( + [ + ("some_list", pa.large_list(pa.string())), + ("some_string", pa.large_string()), + ("some_binary", pa.large_binary()), + ] + ), + pa.schema( + [ + ("some_list", pa.list_(pa.string())), + ("some_string", pa.string()), + ("some_binary", pa.binary()), + ] + ), + False, + ), + ( + pa.schema( + [ + ("highly_nested_list", pa.list_(pa.list_(pa.list_(pa.string())))), + ( + "highly_nested_list_binary", + pa.list_(pa.list_(pa.list_(pa.binary()))), + ), + ("some_string", pa.large_string()), + ("some_binary", pa.large_binary()), + ] + ), + pa.schema( + [ + ( + "highly_nested_list", + pa.large_list(pa.large_list(pa.large_list(pa.large_string()))), + ), + ( + "highly_nested_list_binary", + pa.large_list(pa.large_list(pa.large_list(pa.large_binary()))), + ), + ("some_string", pa.large_string()), + ("some_binary", pa.large_binary()), + ] + ), + True, + ), + ( + pa.schema( + [ + ( + "highly_nested_list", + pa.large_list(pa.list_(pa.large_list(pa.string()))), + ), + ( + "highly_nested_list_int", + pa.large_list(pa.list_(pa.large_list(pa.uint64()))), + ), + ("some_string", pa.large_string()), + ("some_binary", pa.large_binary()), + ] + ), + pa.schema( + [ + ("highly_nested_list", pa.list_(pa.list_(pa.list_(pa.string())))), + ( + "highly_nested_list_int", + pa.list_(pa.list_(pa.list_(pa.int64()))), + ), + ("some_string", pa.string()), + ("some_binary", pa.binary()), + ] + ), + False, + ), + ( + pa.schema( + [ + ("timestamp", pa.timestamp("s")), + ("timestamp1", pa.timestamp("ms")), + ("timestamp2", pa.timestamp("us")), + ("timestamp3", pa.timestamp("ns")), + ("timestamp4", pa.timestamp("s", tz="UTC")), + ("timestamp5", pa.timestamp("ms", tz="UTC")), + ("timestamp6", pa.timestamp("ns", tz="UTC")), + ("timestamp7", pa.timestamp("ns", tz="UTC")), + ] + ), + pa.schema( + [ + ("timestamp", pa.timestamp("us")), + ("timestamp1", pa.timestamp("us")), + ("timestamp2", pa.timestamp("us")), + ("timestamp3", pa.timestamp("us")), + ("timestamp4", pa.timestamp("us")), + ("timestamp5", pa.timestamp("us")), + ("timestamp6", pa.timestamp("us")), + ("timestamp7", pa.timestamp("us")), + ] + ), + False, + ), + ( + pa.schema( + [ + ( + "struct", + pa.struct( + { + "highly_nested_list": pa.large_list( + pa.list_(pa.large_list(pa.string())) + ), + "highly_nested_list_int": pa.large_list( + pa.list_(pa.large_list(pa.uint64())) + ), + "some_string": pa.large_string(), + "some_binary": pa.large_binary(), + } + ), + ) + ] + ), + pa.schema( + [ + ( + "struct", + pa.struct( + { + "highly_nested_list": pa.list_( + pa.list_(pa.list_(pa.string())) + ), + "highly_nested_list_int": pa.list_( + pa.list_(pa.list_(pa.int64())) + ), + "some_string": pa.string(), + "some_binary": pa.binary(), + } + ), + ) + ] + ), + False, + ), + ], +) +def test_schema_conversions(schema, expected_schema, large_dtypes): + result_schema = _convert_pa_schema_to_delta(schema, large_dtypes) + + assert result_schema == expected_schema diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 3de0bc2749..343c098054 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -419,11 +419,39 @@ def test_write_iterator( assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +@pytest.mark.parametrize("large_dtypes", [True, False]) +@pytest.mark.parametrize( + "constructor", + [ + lambda table: table.to_pyarrow_dataset(), + lambda table: table.to_pyarrow_table(), + lambda table: table.to_pyarrow_table().to_batches()[0], + ], +) +def test_write_dataset_table_recordbatch( + tmp_path: pathlib.Path, + existing_table: DeltaTable, + sample_data: pa.Table, + engine: str, + large_dtypes: bool, + constructor, +): + dataset = constructor(existing_table) + + write_deltalake(tmp_path, dataset, mode="overwrite", large_dtypes=large_dtypes, engine=engine) + assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data + + +@pytest.mark.parametrize("large_dtypes", [True, False]) @pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_write_recordbatchreader( tmp_path: pathlib.Path, + existing_table: DeltaTable, + sample_data: pa.Table, + large_dtypes: bool,, engine: Literal["pyarrow", "rust"], ): batches = existing_table.to_pyarrow_dataset().to_batches() @@ -431,7 +459,7 @@ def test_write_recordbatchreader( existing_table.to_pyarrow_dataset().schema, batches ) - write_deltalake(tmp_path, reader, mode="overwrite", engine=engine) + write_deltalake(tmp_path, reader, mode="overwrite", large_dtypes=large_dtypes, engine=engine) assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data From 573e8fea20f05ee57377bbc9782041239ce2a4be Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 25 Nov 2023 09:50:57 +0100 Subject: [PATCH 30/35] resolve merge --- python/deltalake/writer.py | 90 +++++++++++-------------------------- python/tests/test_writer.py | 12 ++--- 2 files changed, 34 insertions(+), 68 deletions(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 7d6719985d..251c176b2d 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -172,12 +172,6 @@ def write_deltalake( storage_options = table._storage_options or {} storage_options.update(storage_options or {}) - __enforce_append_only(table=table, configuration=configuration, mode=mode) - - if isinstance(partition_by, str): - partition_by = [partition_by] - - if table: table.update_incremental() if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: raise DeltaProtocolError( @@ -185,60 +179,11 @@ def write_deltalake( f"{table.protocol().min_writer_version}, " "but this method only supports version 2." ) - if engine == "rust": - if table is not None and mode == "ignore": - return - - if isinstance(data, RecordBatchReader): - batch_iter = data - elif isinstance(data, pa.RecordBatch): - batch_iter = [data] - elif isinstance(data, pa.Table): - batch_iter = data.to_reader() - elif isinstance(data, ds.Dataset): - batch_iter = data.scanner().to_reader() - elif _has_pandas and isinstance(data, pd.DataFrame): - if schema is not None: - batch_iter = pa.Table.from_pandas(data, schema).to_reader() - else: - batch_iter = pa.Table.from_pandas(data).to_reader() - else: - batch_iter = data - - if schema is None: - if isinstance(batch_iter, RecordBatchReader): - schema = batch_iter.schema - elif isinstance(batch_iter, Iterable): - raise ValueError("You must provide schema if data is Iterable") - - data = RecordBatchReader.from_batches(schema, (batch for batch in batch_iter)) - write_deltalake_rust( - table_uri=table_uri, - data=data, - partition_by=partition_by, - mode=mode, - max_rows_per_group=max_rows_per_group, - overwrite_schema=overwrite_schema, - name=name, - description=description, - configuration=configuration, - storage_options=storage_options, - ) - if table: - table.update_incremental() - elif engine == "pyarrow": - if _has_pandas and isinstance(data, pd.DataFrame): - if schema is not None: - data = pa.Table.from_pandas(data, schema=schema) - else: - data, schema = delta_arrow_schema_from_pandas(data) - - table, table_uri = try_get_table_and_table_uri(table_or_uri, storage_options) + __enforce_append_only(table=table, configuration=configuration, mode=mode) - # We need to write against the latest table version - if table: - table.update_incremental() + if isinstance(partition_by, str): + partition_by = [partition_by] if isinstance(data, RecordBatchReader): data = convert_pyarrow_recordbatchreader(data, large_dtypes) @@ -261,9 +206,28 @@ def write_deltalake( f"{type(data).__name__} is not a valid input. Only PyArrow RecordBatchReader, RecordBatch, Iterable[RecordBatch], Table, Dataset or Pandas DataFrame are valid inputs for source." ) - if schema is None: - schema = data.schema + if engine == "rust": + if table is not None and mode == "ignore": + return + data = RecordBatchReader.from_batches(schema, (batch for batch in data)) + write_deltalake_rust( + table_uri=table_uri, + data=data, + partition_by=partition_by, + mode=mode, + max_rows_per_group=max_rows_per_group, + overwrite_schema=overwrite_schema, + name=name, + description=description, + configuration=configuration, + storage_options=storage_options, + ) + if table: + table.update_incremental() + + elif engine == "pyarrow": + # We need to write against the latest table version if filesystem is not None: raise NotImplementedError( "Filesystem support is not yet implemented. #570" @@ -399,9 +363,9 @@ def validate_batch(batch: pa.RecordBatch) -> pa.RecordBatch: return batch - data = RecordBatchReader.from_batches( - schema, (validate_batch(batch) for batch in data) - ) + data = RecordBatchReader.from_batches( + schema, (validate_batch(batch) for batch in data) + ) if file_options is not None: file_options.update(use_compliant_nested_type=False) diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 343c098054..587b2da43f 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -439,7 +439,9 @@ def test_write_dataset_table_recordbatch( ): dataset = constructor(existing_table) - write_deltalake(tmp_path, dataset, mode="overwrite", large_dtypes=large_dtypes, engine=engine) + write_deltalake( + tmp_path, dataset, mode="overwrite", large_dtypes=large_dtypes, engine=engine + ) assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data @@ -447,11 +449,9 @@ def test_write_dataset_table_recordbatch( @pytest.mark.parametrize("engine", ["pyarrow", "rust"]) def test_write_recordbatchreader( tmp_path: pathlib.Path, - existing_table: DeltaTable, - sample_data: pa.Table, - large_dtypes: bool,, + large_dtypes: bool, engine: Literal["pyarrow", "rust"], ): batches = existing_table.to_pyarrow_dataset().to_batches() @@ -459,7 +459,9 @@ def test_write_recordbatchreader( existing_table.to_pyarrow_dataset().schema, batches ) - write_deltalake(tmp_path, reader, mode="overwrite", large_dtypes=large_dtypes, engine=engine) + write_deltalake( + tmp_path, reader, mode="overwrite", large_dtypes=large_dtypes, engine=engine + ) assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data From bb5815a0da679a6be157e456e14fdce9bc7a6d43 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 25 Nov 2023 10:25:18 +0100 Subject: [PATCH 31/35] fix bug from conflict --- python/deltalake/writer.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 251c176b2d..a8cb91b846 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -195,9 +195,13 @@ def write_deltalake( data = convert_pyarrow_dataset(data, large_dtypes) elif _has_pandas and isinstance(data, pd.DataFrame): if schema is not None: - data = pa.Table.from_pandas(data, schema=schema) + data = convert_pyarrow_table( + pa.Table.from_pandas(data, schema=schema), large_dtypes=large_dtypes + ) else: - data = convert_pyarrow_table(pa.Table.from_pandas(data), False) + data = convert_pyarrow_table( + pa.Table.from_pandas(data), large_dtypes=large_dtypes + ) elif isinstance(data, Iterable): if schema is None: raise ValueError("You must provide schema if data is Iterable") @@ -206,6 +210,9 @@ def write_deltalake( f"{type(data).__name__} is not a valid input. Only PyArrow RecordBatchReader, RecordBatch, Iterable[RecordBatch], Table, Dataset or Pandas DataFrame are valid inputs for source." ) + if schema is None: + schema = data.schema + if engine == "rust": if table is not None and mode == "ignore": return From 381df0d79d5bdc535cd2cc1bf2b989fecaa285e3 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 25 Nov 2023 10:48:03 +0100 Subject: [PATCH 32/35] remove commented code --- crates/deltalake-core/src/operations/write.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index c868048b99..43df84a767 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -491,7 +491,6 @@ impl std::future::IntoFuture for WriteBuilder { .or_else(|_| this.snapshot.arrow_schema()) .unwrap_or(schema.clone()); - // let data_schema: StructType = schema.clone().try_into().unwrap(); if schema != table_schema { let mut metadata = this .snapshot From 3ab76871f5b4a92a990109a16aa9e3ad01e34fbe Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 25 Nov 2023 12:03:55 +0100 Subject: [PATCH 33/35] expose predicate, handle error better, and add overloads --- crates/deltalake-core/src/operations/write.rs | 4 +- python/deltalake/_internal.pyi | 1 + python/deltalake/writer.py | 66 ++++++++++++++++++ python/src/lib.rs | 5 ++ python/tests/test_writer.py | 68 +++++++++++++++++++ 5 files changed, 143 insertions(+), 1 deletion(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index 43df84a767..b005284f70 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -525,7 +525,9 @@ impl std::future::IntoFuture for WriteBuilder { match this.predicate { Some(_pred) => { - todo!("Overwriting data based on predicate is not yet implemented") + return Err(DeltaTableError::Generic( + "Overwriting data based on predicate is not yet implemented".to_string(), + )); } _ => { let remove_actions = this diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index 854d96d4a8..d7c0e1a8f9 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -147,6 +147,7 @@ def write_to_deltalake( mode: str, max_rows_per_group: int, overwrite_schema: bool, + predicate: Optional[str], name: Optional[str], description: Optional[str], configuration: Optional[Mapping[str, Optional[str]]], diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index a8cb91b846..1b3ea72668 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -17,6 +17,7 @@ Optional, Tuple, Union, + overload, ) from urllib.parse import unquote @@ -68,6 +69,68 @@ class AddAction: stats: str +@overload +def write_deltalake( + table_or_uri: Union[str, Path, DeltaTable], + data: Union[ + "pd.DataFrame", + ds.Dataset, + pa.Table, + pa.RecordBatch, + Iterable[pa.RecordBatch], + RecordBatchReader, + ], + *, + schema: Optional[pa.Schema] = ..., + partition_by: Optional[Union[List[str], str]] = ..., + filesystem: Optional[pa_fs.FileSystem] = None, + mode: Literal["error", "append", "overwrite", "ignore"] = ..., + file_options: Optional[ds.ParquetFileWriteOptions] = ..., + max_partitions: Optional[int] = ..., + max_open_files: int = ..., + max_rows_per_file: int = ..., + min_rows_per_group: int = ..., + max_rows_per_group: int = ..., + name: Optional[str] = ..., + description: Optional[str] = ..., + configuration: Optional[Mapping[str, Optional[str]]] = ..., + overwrite_schema: bool = ..., + storage_options: Optional[Dict[str, str]] = ..., + partition_filters: Optional[List[Tuple[str, str, Any]]] = ..., + large_dtypes: bool = ..., + engine: Literal["pyarrow"] = ..., +) -> None: + ... + + +@overload +def write_deltalake( + table_or_uri: Union[str, Path, DeltaTable], + data: Union[ + "pd.DataFrame", + ds.Dataset, + pa.Table, + pa.RecordBatch, + Iterable[pa.RecordBatch], + RecordBatchReader, + ], + *, + schema: Optional[pa.Schema] = ..., + partition_by: Optional[Union[List[str], str]] = ..., + mode: Literal["error", "append", "overwrite", "ignore"] = ..., + max_rows_per_group: int = ..., + name: Optional[str] = ..., + description: Optional[str] = ..., + configuration: Optional[Mapping[str, Optional[str]]] = ..., + overwrite_schema: bool = ..., + storage_options: Optional[Dict[str, str]] = ..., + predicate: Optional[str] = ..., + large_dtypes: bool = ..., + engine: Literal["rust"], +) -> None: + ... + + def write_deltalake( table_or_uri: Union[str, Path, DeltaTable], data: Union[ @@ -95,6 +158,7 @@ def write_deltalake( overwrite_schema: bool = False, storage_options: Optional[Dict[str, str]] = None, partition_filters: Optional[List[Tuple[str, str, Any]]] = None, + predicate: Optional[str] = None, large_dtypes: bool = False, engine: Literal["pyarrow", "rust"] = "pyarrow", ) -> None: @@ -164,6 +228,7 @@ def write_deltalake( configuration: A map containing configuration options for the metadata action. overwrite_schema: If True, allows updating the schema of the table. storage_options: options passed to the native delta filesystem. Unused if 'filesystem' is defined. + predicate: When using `Overwrite` mode, replace data that matches a predicate. Only used in rust engine. partition_filters: the partition filters that will be used for partition overwrite. Only used in pyarrow engine. large_dtypes: If True, the data schema is kept in large_dtypes, has no effect on pandas dataframe input """ @@ -225,6 +290,7 @@ def write_deltalake( mode=mode, max_rows_per_group=max_rows_per_group, overwrite_schema=overwrite_schema, + predicate=predicate, name=name, description=description, configuration=configuration, diff --git a/python/src/lib.rs b/python/src/lib.rs index be0550ff4d..e7d5ec818d 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1135,6 +1135,7 @@ fn write_to_deltalake( max_rows_per_group: i64, overwrite_schema: bool, partition_by: Option>, + predicate: Option, name: Option, description: Option, configuration: Option>>, @@ -1168,6 +1169,10 @@ fn write_to_deltalake( builder = builder.with_description(description); }; + if let Some(predicate) = &predicate { + builder = builder.with_replace_where(predicate); + }; + if let Some(config) = configuration { builder = builder.with_configuration(config); }; diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 587b2da43f..4c61155e76 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -854,6 +854,74 @@ def test_partition_overwrite_unfiltered_data_fails( ) +@pytest.mark.parametrize( + "value_1,value_2,value_type,filter_string", + [ + (1, 2, pa.int64(), "1"), + (False, True, pa.bool_(), "false"), + (date(2022, 1, 1), date(2022, 1, 2), pa.date32(), "2022-01-01"), + ], +) +def test_replace_where_overwrite( + tmp_path: pathlib.Path, + value_1: Any, + value_2: Any, + value_type: pa.DataType, + filter_string: str, +): + sample_data = pa.table( + { + "p1": pa.array(["1", "1", "2", "2"], pa.string()), + "p2": pa.array([value_1, value_2, value_1, value_2], value_type), + "val": pa.array([1, 1, 1, 1], pa.int64()), + } + ) + write_deltalake(tmp_path, sample_data, mode="overwrite", partition_by=["p1", "p2"]) + + delta_table = DeltaTable(tmp_path) + assert ( + delta_table.to_pyarrow_table().sort_by( + [("p1", "ascending"), ("p2", "ascending")] + ) + == sample_data + ) + + sample_data = pa.table( + { + "p1": pa.array(["1", "1"], pa.string()), + "p2": pa.array([value_2, value_1], value_type), + "val": pa.array([2, 2], pa.int64()), + } + ) + expected_data = pa.table( + { + "p1": pa.array(["1", "1", "2", "2"], pa.string()), + "p2": pa.array([value_1, value_2, value_1, value_2], value_type), + "val": pa.array([2, 2, 1, 1], pa.int64()), + } + ) + + with pytest.raises( + DeltaError, + match="Generic DeltaTable error: Overwriting data based on predicate is not yet implemented", + ): + write_deltalake( + tmp_path, + sample_data, + mode="overwrite", + predicate="`p1` = 1", + engine="rust", + ) + + delta_table.update_incremental() + assert ( + delta_table.to_pyarrow_table().sort_by( + [("p1", "ascending"), ("p2", "ascending")] + ) + == expected_data + ) + + def test_partition_overwrite_with_new_partition( tmp_path: pathlib.Path, sample_data_for_partitioning: pa.Table ): From e73eea3b1022174c102658d23740fcdfb175fa68 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 25 Nov 2023 12:08:56 +0100 Subject: [PATCH 34/35] fmt --- crates/deltalake-core/src/operations/write.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index b005284f70..9fbeecf439 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -526,7 +526,8 @@ impl std::future::IntoFuture for WriteBuilder { match this.predicate { Some(_pred) => { return Err(DeltaTableError::Generic( - "Overwriting data based on predicate is not yet implemented".to_string(), + "Overwriting data based on predicate is not yet implemented" + .to_string(), )); } _ => { From b4f9695ac3cc7e840434d25010e495bb93a21415 Mon Sep 17 00:00:00 2001 From: ion-elgreco <15728914+ion-elgreco@users.noreply.github.com> Date: Sat, 25 Nov 2023 16:14:48 +0100 Subject: [PATCH 35/35] use ? instead of unwrap --- crates/deltalake-core/src/operations/write.rs | 4 ++-- python/deltalake/writer.py | 13 +++++++------ python/tests/test_writer.py | 2 +- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index 9fbeecf439..8fd8ddd99e 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -497,8 +497,8 @@ impl std::future::IntoFuture for WriteBuilder { .current_metadata() .ok_or(DeltaTableError::NoMetadata)? .clone(); - metadata.schema = schema.clone().try_into().unwrap(); - let metadata_action = Metadata::try_from(metadata).unwrap(); + metadata.schema = schema.clone().try_into()?; + let metadata_action = Metadata::try_from(metadata)?; actions.push(Action::Metadata(metadata_action)); } // This should never error, since now() will always be larger than UNIX_EPOCH diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 1b3ea72668..626fb1a5d9 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -238,12 +238,6 @@ def write_deltalake( storage_options.update(storage_options or {}) table.update_incremental() - if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: - raise DeltaProtocolError( - "This table's min_writer_version is " - f"{table.protocol().min_writer_version}, " - "but this method only supports version 2." - ) __enforce_append_only(table=table, configuration=configuration, mode=mode) @@ -385,6 +379,13 @@ def visitor(written_file: Any) -> None: if table is not None: # We don't currently provide a way to set invariants # (and maybe never will), so only enforce if already exist. + if table.protocol().min_writer_version > MAX_SUPPORTED_WRITER_VERSION: + raise DeltaProtocolError( + "This table's min_writer_version is " + f"{table.protocol().min_writer_version}, " + "but this method only supports version 2." + ) + invariants = table.schema().invariants checker = _DeltaDataChecker(invariants) diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 4c61155e76..0a63b16c70 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -581,7 +581,7 @@ def test_writer_null_stats(tmp_path: pathlib.Path, engine: Literal["pyarrow", "r assert stats["nullCount"] == expected_nulls -@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +@pytest.mark.parametrize("engine", ["pyarrow"]) def test_writer_fails_on_protocol( existing_table: DeltaTable, sample_data: pa.Table,