|
| 1 | +"""The `daft.catalog` module contains functionality for Data Catalogs. |
| 2 | +
|
| 3 | +A Data Catalog can be understood as a system/service for users to discover, access and query their data. |
| 4 | +Most commonly, users' data is represented as a "table". Some more modern Data Catalogs such as Unity Catalog |
| 5 | +also expose other types of data including files, ML models, registered functions and more. |
| 6 | +
|
| 7 | +Examples of Data Catalogs include AWS Glue, Hive Metastore, Apache Iceberg REST and Unity Catalog. |
| 8 | +
|
| 9 | +Daft manages Data Catalogs by registering them in an internal meta-catalog, called the "DaftMetaCatalog". This |
| 10 | +is simple a collection of data catalogs, which Daft will attempt to detect from a users' current environment. |
| 11 | +
|
| 12 | +**Data Catalog** |
| 13 | +
|
| 14 | +Daft recognizes a default catalog which it will attempt to use when no specific catalog name is provided. |
| 15 | +
|
| 16 | +```python |
| 17 | +# This will hit the default catalog |
| 18 | +daft.read_table("my_db.my_namespace.my_table") |
| 19 | +``` |
| 20 | +
|
| 21 | +**Named Tables** |
| 22 | +
|
| 23 | +Daft allows also the registration of named tables, which have no catalog associated with them. |
| 24 | +
|
| 25 | +Note that named tables take precedence over the default catalog's table names when resolving names. |
| 26 | +
|
| 27 | +```python |
| 28 | +df = daft.from_pydict({"foo": [1, 2, 3]}) |
| 29 | +
|
| 30 | +daft.catalog.register_named_table( |
| 31 | + "my_table", |
| 32 | + df, |
| 33 | +) |
| 34 | +
|
| 35 | +# Your table is now accessible from Daft-SQL, or Daft's `read_table` |
| 36 | +df1 = daft.read_table("my_table") |
| 37 | +df2 = daft.sql("SELECT * FROM my_table") |
| 38 | +``` |
| 39 | +""" |
| 40 | + |
| 41 | +from __future__ import annotations |
| 42 | + |
| 43 | +from daft.daft import catalog as native_catalog |
| 44 | +from daft.logical.builder import LogicalPlanBuilder |
| 45 | + |
| 46 | +from daft.dataframe import DataFrame |
| 47 | + |
| 48 | +_PYICEBERG_AVAILABLE = False |
| 49 | +try: |
| 50 | + from pyiceberg.catalog import Catalog as PyIcebergCatalog |
| 51 | + |
| 52 | + _PYICEBERG_AVAILABLE = True |
| 53 | +except ImportError: |
| 54 | + pass |
| 55 | + |
| 56 | +_UNITY_AVAILABLE = False |
| 57 | +try: |
| 58 | + from daft.unity_catalog import UnityCatalog |
| 59 | + |
| 60 | + _UNITY_AVAILABLE = True |
| 61 | +except ImportError: |
| 62 | + pass |
| 63 | + |
| 64 | +__all__ = [ |
| 65 | + "read_table", |
| 66 | + "register_python_catalog", |
| 67 | + "unregister_catalog", |
| 68 | + "register_table", |
| 69 | +] |
| 70 | + |
| 71 | +# Forward imports from the native catalog which don't require Python wrappers |
| 72 | +unregister_catalog = native_catalog.unregister_catalog |
| 73 | + |
| 74 | + |
| 75 | +def read_table(name: str) -> DataFrame: |
| 76 | + """Finds a table with the specified name and reads it as a DataFrame |
| 77 | +
|
| 78 | + The provided name can be any of the following, and Daft will return them with the following order of priority: |
| 79 | +
|
| 80 | + 1. Name of a registered dataframe/SQL view (manually registered using `daft.register_table`): `"my_registered_table"` |
| 81 | + 2. Name of a table within the default catalog (without inputting the catalog name) for example: `"my.table.name"` |
| 82 | + 3. Name of a fully-qualified table path with the catalog name for example: `"my_catalog.my.table.name"` |
| 83 | +
|
| 84 | + Args: |
| 85 | + name: The identifier for the table to read |
| 86 | +
|
| 87 | + Returns: |
| 88 | + A DataFrame containing the data from the specified table. |
| 89 | + """ |
| 90 | + native_logical_plan_builder = native_catalog.read_table(name) |
| 91 | + return DataFrame(LogicalPlanBuilder(native_logical_plan_builder)) |
| 92 | + |
| 93 | + |
| 94 | +def register_table(name: str, dataframe: DataFrame) -> str: |
| 95 | + """Register a DataFrame as a named table. |
| 96 | +
|
| 97 | + This function registers a DataFrame as a named table, making it accessible |
| 98 | + via Daft-SQL or Daft's `read_table` function. |
| 99 | +
|
| 100 | + Args: |
| 101 | + name (str): The name to register the table under. |
| 102 | + dataframe (daft.DataFrame): The DataFrame to register as a table. |
| 103 | +
|
| 104 | + Returns: |
| 105 | + str: The name of the registered table. |
| 106 | +
|
| 107 | + Example: |
| 108 | + >>> df = daft.from_pydict({"foo": [1, 2, 3]}) |
| 109 | + >>> daft.catalog.register_table("my_table", df) |
| 110 | + >>> daft.read_table("my_table") |
| 111 | + """ |
| 112 | + return native_catalog.register_table(name, dataframe._builder._builder) |
| 113 | + |
| 114 | + |
| 115 | +def register_python_catalog(catalog: PyIcebergCatalog | UnityCatalog, name: str | None = None) -> str: |
| 116 | + """Registers a Python catalog with Daft |
| 117 | +
|
| 118 | + Currently supports: |
| 119 | +
|
| 120 | + * [PyIceberg Catalogs](https://py.iceberg.apache.org/api/) |
| 121 | + * [Unity Catalog](https://www.getdaft.io/projects/docs/en/latest/user_guide/integrations/unity-catalog.html) |
| 122 | +
|
| 123 | + Args: |
| 124 | + catalog (PyIcebergCatalog | UnityCatalog): The Python catalog to register. |
| 125 | + name (str | None, optional): The name to register the catalog under. If None, this catalog is registered as the default catalog. |
| 126 | +
|
| 127 | + Returns: |
| 128 | + str: The name of the registered catalog. |
| 129 | +
|
| 130 | + Raises: |
| 131 | + ValueError: If an unsupported catalog type is provided. |
| 132 | +
|
| 133 | + Example: |
| 134 | + >>> from pyiceberg.catalog import load_catalog |
| 135 | + >>> catalog = load_catalog("my_catalog") |
| 136 | + >>> daft.catalog.register_python_catalog(catalog, "my_daft_catalog") |
| 137 | +
|
| 138 | + """ |
| 139 | + python_catalog: PyIcebergCatalog |
| 140 | + if _PYICEBERG_AVAILABLE and isinstance(catalog, PyIcebergCatalog): |
| 141 | + from daft.catalog.pyiceberg import PyIcebergCatalogAdaptor |
| 142 | + |
| 143 | + python_catalog = PyIcebergCatalogAdaptor(catalog) |
| 144 | + elif _UNITY_AVAILABLE and isinstance(catalog, UnityCatalog): |
| 145 | + from daft.catalog.unity import UnityCatalogAdaptor |
| 146 | + |
| 147 | + python_catalog = UnityCatalogAdaptor(catalog) |
| 148 | + else: |
| 149 | + raise ValueError(f"Unsupported catalog type: {type(catalog)}") |
| 150 | + |
| 151 | + return native_catalog.register_python_catalog(python_catalog, name) |
0 commit comments