Skip to content

Commit

Permalink
doc update for tag
Browse files Browse the repository at this point in the history
  • Loading branch information
deltars committed Oct 18, 2023
1 parent 5eff692 commit a97d789
Show file tree
Hide file tree
Showing 10 changed files with 649 additions and 113 deletions.
153 changes: 118 additions & 35 deletions python/_sources/usage.rst.txt
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,8 @@ of the table, and other metadata such as creation time.
>>> dt.version()
3
>>> dt.files()
['part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet',
'part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet',
['part-00000-cb6b150b-30b8-4662-ad28-ff32ddab96d2-c000.snappy.parquet',
'part-00000-7c2deba3-1994-4fb8-bc07-d46c948aa415-c000.snappy.parquet',
'part-00001-c373a5bd-85f0-4758-815e-7eb62007a15c-c000.snappy.parquet']
Expand Down Expand Up @@ -54,7 +54,7 @@ being used. We try to support many of the well-known formats to identify basic s

* gs://<bucket>/<path>

Alternatively, if you have a data catalog you can load it by reference to a
Alternatively, if you have a data catalog you can load it by reference to a
database and table name. Currently supported are AWS Glue and Databricks Unity Catalog.

For AWS Glue catalog, use AWS environment variables to authenticate.
Expand All @@ -76,12 +76,15 @@ For Databricks Unity Catalog authentication, use environment variables:

.. code-block:: python
>>> from deltalake import DataCatalog, DeltaTable
>>> catalog_name = 'main'
>>> schema_name = 'db_schema'
>>> table_name = 'db_table'
>>> data_catalog = DataCatalog.UNITY
>>> dt = DeltaTable.from_data_catalog(data_catalog=data_catalog, data_catalog_id=catalog_name, database_name=schema_name, table_name=table_name)
>>> import os
>>> from deltalake import DataCatalog, DeltaTable
>>> os.environ['DATABRICKS_WORKSPACE_URL'] = "https://adb-62800498333851.30.azuredatabricks.net"
>>> os.environ['DATABRICKS_ACCESS_TOKEN'] = "<DBAT>"
>>> catalog_name = 'main'
>>> schema_name = 'db_schema'
>>> table_name = 'db_table'
>>> data_catalog = DataCatalog.UNITY
>>> dt = DeltaTable.from_data_catalog(data_catalog=data_catalog, data_catalog_id=catalog_name, database_name=schema_name, table_name=table_name)
.. _`s3 options`: https://docs.rs/object_store/latest/object_store/aws/enum.AmazonS3ConfigKey.html#variants
.. _`azure options`: https://docs.rs/object_store/latest/object_store/azure/enum.AzureConfigKey.html#variants
Expand All @@ -92,7 +95,7 @@ Custom Storage Backends

While delta always needs its internal storage backend to work and be properly configured, in order to manage the delta log,
it may sometime be advantageous - and is common practice in the arrow world - to customize the storage interface used for
reading the bulk data.
reading the bulk data.

``deltalake`` will work with any storage compliant with :class:`pyarrow.fs.FileSystem`, however the root of the filesystem has
to be adjusted to point at the root of the Delta table. We can achieve this by wrapping the custom filesystem into
Expand All @@ -102,10 +105,10 @@ a :class:`pyarrow.fs.SubTreeFileSystem`.
import pyarrow.fs as fs
from deltalake import DeltaTable
path = "<path/to/table>"
filesystem = fs.SubTreeFileSystem(path, fs.LocalFileSystem())
dt = DeltaTable(path)
ds = dt.to_pyarrow_dataset(filesystem=filesystem)
Expand Down Expand Up @@ -176,8 +179,8 @@ Schema
~~~~~~

The schema for the table is also saved in the transaction log. It can either be
retrieved in the Delta Lake form as :class:`deltalake.schema.Schema` or as a PyArrow
schema. The first allows you to introspect any column-level metadata stored in
retrieved in the Delta Lake form as :class:`deltalake.schema.Schema` or as a PyArrow
schema. The first allows you to introspect any column-level metadata stored in
the schema, while the latter represents the schema the table will be loaded into.

Use :meth:`DeltaTable.schema` to retrieve the delta lake schema:
Expand Down Expand Up @@ -209,14 +212,14 @@ History
~~~~~~~

Depending on what system wrote the table, the delta table may have provenance
information describing what operations were performed on the table, when, and
information describing what operations were performed on the table, when, and
by whom. This information is retained for 30 days by default, unless otherwise
specified by the table configuration ``delta.logRetentionDuration``.

.. note::

This information is not written by all writers and different writers may use
different schemas to encode the actions. For Spark's format, see:
This information is not written by all writers and different writers may use
different schemas to encode the actions. For Spark's format, see:
https://docs.delta.io/latest/delta-utility.html#history-schema

To view the available history, use :meth:`DeltaTable.history`:
Expand All @@ -236,7 +239,7 @@ To view the available history, use :meth:`DeltaTable.history`:
Current Add Actions
~~~~~~~~~~~~~~~~~~~

The active state for a delta table is determined by the Add actions, which
The active state for a delta table is determined by the Add actions, which
provide the list of files that are part of the table and metadata about them,
such as creation time, size, and statistics. You can get a data frame of
the add actions data using :meth:`DeltaTable.get_add_actions`:
Expand Down Expand Up @@ -265,18 +268,18 @@ Querying Delta Tables
---------------------

Delta tables can be queried in several ways. By loading as Arrow data or an Arrow
dataset, they can be used by compatible engines such as Pandas and DuckDB. By
dataset, they can be used by compatible engines such as Pandas and DuckDB. By
passing on the list of files, they can be loaded into other engines such as Dask.

Delta tables are often larger than can fit into memory on a single computer, so
this module provides ways to read only the parts of the data you need. Partition
this module provides ways to read only the parts of the data you need. Partition
filters allow you to skip reading files that are part of irrelevant partitions.
Only loading the columns required also saves memory. Finally, some methods allow
reading tables batch-by-batch, allowing you to process the whole table while only
having a portion loaded at any given time.

To load into Pandas or a PyArrow table use the :meth:`DeltaTable.to_pandas` and
:meth:`DeltaTable.to_pyarrow_table` methods, respectively. Both of these
:meth:`DeltaTable.to_pyarrow_table` methods, respectively. Both of these
support filtering partitions and selecting particular columns.

.. code-block:: python
Expand All @@ -298,10 +301,10 @@ support filtering partitions and selecting particular columns.
pyarrow.Table
value: string
Converting to a PyArrow Dataset allows you to filter on columns other than
Converting to a PyArrow Dataset allows you to filter on columns other than
partition columns and load the result as a stream of batches rather than a single
table. Convert to a dataset using :meth:`DeltaTable.to_pyarrow_dataset`. Filters
applied to datasets will use the partition values and file statistics from the
table. Convert to a dataset using :meth:`DeltaTable.to_pyarrow_dataset`. Filters
applied to datasets will use the partition values and file statistics from the
Delta transaction log and push down any other filters to the scanning operation.

.. code-block:: python
Expand Down Expand Up @@ -363,7 +366,7 @@ you can pass them to ``dask.dataframe.read_parquet``:
>>> df
Dask DataFrame Structure:
value year month day
npartitions=6
npartitions=6
object category[known] category[known] category[known]
... ... ... ...
... ... ... ... ...
Expand All @@ -388,7 +391,7 @@ Vacuuming tables
~~~~~~~~~~~~~~~~

Vacuuming a table will delete any files that have been marked for deletion. This
may make some past versions of a table invalid, so this can break time travel.
may make some past versions of a table invalid, so this can break time travel.
However, it will save storage space. Vacuum will retain files in a certain window,
by default one week, so time travel will still work in shorter ranges.

Expand All @@ -403,8 +406,8 @@ only list the files to be deleted. Pass ``dry_run=False`` to actually delete fil
>>> dt = DeltaTable("../rust/tests/data/simple_table")
>>> dt.vacuum()
['../rust/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet',
'../rust/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet',
['../rust/tests/data/simple_table/part-00006-46f2ff20-eb5d-4dda-8498-7bfb2940713b-c000.snappy.parquet',
'../rust/tests/data/simple_table/part-00190-8ac0ae67-fb1d-461d-a3d3-8dc112766ff5-c000.snappy.parquet',
'../rust/tests/data/simple_table/part-00164-bf40481c-4afd-4c02-befa-90f056c2d77a-c000.snappy.parquet',
...]
>>> dt.vacuum(dry_run=False) # Don't run this unless you are sure!
Expand Down Expand Up @@ -458,16 +461,16 @@ DataFrame, a PyArrow Table, or an iterator of PyArrow Record Batches.

.. code-block:: python
>>> from deltalake.writer import write_deltalake
>>> from deltalake import write_deltalake
>>> df = pd.DataFrame({'x': [1, 2, 3]})
>>> write_deltalake('path/to/table', df)
.. note::
:py:func:`write_deltalake` accepts a Pandas DataFrame, but will convert it to
a Arrow table before writing. See caveats in :doc:`pyarrow:python/pandas`.
:py:func:`write_deltalake` accepts a Pandas DataFrame, but will convert it to
a Arrow table before writing. See caveats in :doc:`pyarrow:python/pandas`.

By default, writes create a new table and error if it already exists. This is
controlled by the ``mode`` parameter, which mirrors the behavior of Spark's
By default, writes create a new table and error if it already exists. This is
controlled by the ``mode`` parameter, which mirrors the behavior of Spark's
:py:meth:`pyspark.sql.DataFrameWriter.saveAsTable` DataFrame method. To overwrite pass in ``mode='overwrite'`` and
to append pass in ``mode='append'``:

Expand All @@ -477,10 +480,57 @@ to append pass in ``mode='append'``:
>>> write_deltalake('path/to/table', df, mode='append')
:py:meth:`write_deltalake` will raise :py:exc:`ValueError` if the schema of
the data passed to it differs from the existing table's schema. If you wish to
the data passed to it differs from the existing table's schema. If you wish to
alter the schema as part of an overwrite pass in ``overwrite_schema=True``.


Updating Delta Tables
---------------------

.. py:currentmodule:: deltalake.table
Row values in an existing delta table can be updated with the :meth:`DeltaTable.update` command. A update
dictionary has to be passed, where they key is the column you wish to update, and the value is a
Expression in string format.

Update all the rows for the column "processed" to the value True.

.. code-block:: python
>>> from deltalake import write_deltalake, DeltaTable
>>> df = pd.DataFrame({'x': [1, 2, 3], 'deleted': [False, False, False]})
>>> write_deltalake('path/to/table', df)
>>> dt = DeltaTable('path/to/table')
>>> dt.update({"processed": "True"})
>>> dt.to_pandas()
>>> x processed
0 1 True
1 2 True
2 3 True
.. note::
:meth:`DeltaTable.update` predicates and updates are all in string format. The predicates and expressions,
are parsed into Apache Datafusion expressions.

Apply a soft deletion based on a predicate, so update all the rows for the column "deleted" to the value
True where x = 3

.. code-block:: python
>>> from deltalake import write_deltalake, DeltaTable
>>> df = pd.DataFrame({'x': [1, 2, 3], 'deleted': [False, False, False]})
>>> write_deltalake('path/to/table', df)
>>> dt = DeltaTable('path/to/table')
>>> dt.update(
... updates={"deleted": "True"},
... predicate= 'x = 3',
... )
>>> dt.to_pandas()
>>> x deleted
0 1 False
1 2 False
2 3 True
Overwriting a partition
~~~~~~~~~~~~~~~~~~~~~~~

Expand All @@ -492,7 +542,7 @@ the method will raise an error.

.. code-block:: python
>>> from deltalake.writer import write_deltalake
>>> from deltalake import write_deltalake
>>> df = pd.DataFrame({'x': [1, 2, 3], 'y': ['a', 'a', 'b']})
>>> write_deltalake('path/to/table', df, partition_by=['y'])
Expand All @@ -510,6 +560,39 @@ This method could also be used to insert a new partition if one doesn't already
exist, making this operation idempotent.


Removing data
~~~~~~~~~~~~~

.. py:currentmodule:: deltalake.table
You can remove rows from a table with :meth:`DeltaTable.delete`. A SQL where clause can
be provided to only remove some rows. If the clause matches some partition values, then
the files under those partition values will be removed. If the clause matches rows
inside some files, then those files will rewritten without the matched rows. Omitting
the clause will remove all files from the table.

.. code-block:: python
>>> from deltalake import DeltaTable, write_deltalake
>>> df = pd.DataFrame({'a': [1, 2, 3], 'to_delete': [False, False, True]})
>>> write_deltalake('path/to/table', df)
>>> table = DeltaTable('path/to/table')
>>> table.delete(predicate="to_delete = true")
{'num_added_files': 1, 'num_removed_files': 1, 'num_deleted_rows': 1, 'num_copied_rows': 2, 'execution_time_ms': 11081, 'scan_time_ms': 3721, 'rewrite_time_ms': 7}
>>> table.to_pandas()
a to_delete
0 1 False
1 2 False
.. note::

:meth:`DeltaTable.delete` does not delete files from storage but only updates the
table state to one where the deleted rows are no longer present. See
`Vacuuming tables`_ for more information.


Restoring tables
~~~~~~~~~~~~~~~~

Expand Down
Loading

0 comments on commit a97d789

Please sign in to comment.