Skip to content

Catalogs and Tables#

Daft integrates with various catalog implementations using its Catalog and Table interfaces. These are high-level APIs to manage catalog objects (tables and namespaces), while also making it easy to leverage Daft's existing daft.read_ and df.write_ APIs for open table formats like Iceberg and Delta Lake. Learn more about Catalogs & Tables in Daft User Guide.

Catalog #

Interface for Python catalog implementations.

A Catalog is a service for discovering, accessing, and querying tabular and non-tabular data. You can instantiate a Catalog using one of the static from_ methods.

Examples:

1
2
3
4
5
6
7
>>> import daft
>>> from daft.catalog import Catalog
>>>
>>> data = {"users": {"id": [1, 2, 3], "name": ["a", "b", "c"]}}
>>> catalog = Catalog.from_pydict(data)
>>> catalog.list_tables()
['users']

Methods:

Name Description
create_namespace

Creates a namespace in this catalog.

create_namespace_if_not_exists

Creates a namespace in this catalog if it does not already exist.

create_table

Creates a table in this catalog.

create_table_if_not_exists

Creates a table in this catalog if it does not already exist.

drop_namespace
drop_table
from_glue

Creates a Daft Catalog backed by the AWS Glue service, with optional client or session.

from_iceberg

Create a Daft Catalog from a PyIceberg catalog object.

from_pydict

Returns an in-memory catalog from a dictionary of table-like objects.

from_s3tables

Creates a Daft Catalog from S3 Tables bucket ARN, with optional client or session.

from_unity

Create a Daft Catalog from a Unity Catalog client.

get_table

Get a table by its identifier or raises if the table does not exist.

has_namespace

Returns True if the namespace exists, otherwise False.

has_table

Returns True if the table exists, otherwise False.

list_namespaces

List namespaces in the catalog which match the given pattern.

list_tables

List tables in the catalog which match the given pattern.

read_table

Returns the table as a DataFrame or raises an exception if it does not exist.

write_table

Attributes:

Name Type Description
name str

Returns the catalog's name.

name #

name: str

Returns the catalog's name.

create_namespace #

create_namespace(identifier: Identifier | str) -> None

Creates a namespace in this catalog.

Parameters:

Name Type Description Default
identifier Identifier | str

namespace identifier

required
Source code in daft/catalog/__init__.py
326
327
328
329
330
331
332
333
334
335
def create_namespace(self, identifier: Identifier | str) -> None:
    """Creates a namespace in this catalog.

    Args:
        identifier (Identifier | str): namespace identifier
    """
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)

    self._create_namespace(identifier)

create_namespace_if_not_exists #

create_namespace_if_not_exists(
    identifier: Identifier | str,
) -> None

Creates a namespace in this catalog if it does not already exist.

Parameters:

Name Type Description Default
identifier Identifier | str

namespace identifier

required
Source code in daft/catalog/__init__.py
337
338
339
340
341
342
343
344
def create_namespace_if_not_exists(self, identifier: Identifier | str) -> None:
    """Creates a namespace in this catalog if it does not already exist.

    Args:
        identifier (Identifier | str): namespace identifier
    """
    if not self.has_namespace(identifier):
        self.create_namespace(identifier)

create_table #

create_table(
    identifier: Identifier | str,
    source: Schema | DataFrame,
    properties: Properties | None = None,
) -> Table

Creates a table in this catalog.

Parameters:

Name Type Description Default
identifier Identifier | str

table identifier

required
source Schema | DataFrame

table source object such as a Schema or DataFrame.

required

Returns:

Name Type Description
Table Table

new table instance.

Source code in daft/catalog/__init__.py
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
def create_table(
    self,
    identifier: Identifier | str,
    source: Schema | DataFrame,
    properties: Properties | None = None,
) -> Table:
    """Creates a table in this catalog.

    Args:
        identifier (Identifier | str): table identifier
        source (Schema | DataFrame): table source object such as a Schema or DataFrame.

    Returns:
        Table: new table instance.
    """
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)
    schema = source.schema() if isinstance(source, DataFrame) else source

    table = self._create_table(identifier, schema, properties)
    if isinstance(source, DataFrame):
        table.append(source)

    return table

create_table_if_not_exists #

create_table_if_not_exists(
    identifier: Identifier | str,
    source: Schema | DataFrame,
    properties: Properties | None = None,
) -> Table

Creates a table in this catalog if it does not already exist.

Parameters:

Name Type Description Default
identifier Identifier | str

table identifier

required
source Schema | DataFrame

table source object such as a Schema or DataFrame.

required

Returns:

Name Type Description
Table Table

the existing table (if exists) or the new table instance.

Source code in daft/catalog/__init__.py
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
def create_table_if_not_exists(
    self,
    identifier: Identifier | str,
    source: Schema | DataFrame,
    properties: Properties | None = None,
) -> Table:
    """Creates a table in this catalog if it does not already exist.

    Args:
        identifier (Identifier | str): table identifier
        source (Schema | DataFrame): table source object such as a Schema or DataFrame.

    Returns:
        Table: the existing table (if exists) or the new table instance.
    """
    if self.has_table(identifier):
        return self.get_table(identifier)
    else:
        return self.create_table(identifier, source, properties)

drop_namespace #

drop_namespace(identifier: Identifier | str) -> None
Source code in daft/catalog/__init__.py
413
414
415
416
417
def drop_namespace(self, identifier: Identifier | str) -> None:
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)

    self._drop_namespace(identifier)

drop_table #

drop_table(identifier: Identifier | str) -> None
Source code in daft/catalog/__init__.py
419
420
421
422
423
def drop_table(self, identifier: Identifier | str) -> None:
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)

    self._drop_table(identifier)

from_glue #

from_glue(
    name: str,
    client: object | None = None,
    session: object | None = None,
) -> Catalog

Creates a Daft Catalog backed by the AWS Glue service, with optional client or session.

Terms
  • AWS Glue -> Daft Catalog
  • AWS Glue Database -> Daft Namespace
  • AWS Glue Table -> Daft Table

Parameters:

Name Type Description Default
name str

glue database name

required
client object | None

optional boto3 client

None
session object | None

optional boto3 session

None

Returns:

Name Type Description
Catalog Catalog

new daft catalog instance backed by AWS Glue.

Source code in daft/catalog/__init__.py
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
@staticmethod
def from_glue(
    name: str,
    client: object | None = None,
    session: object | None = None,
) -> Catalog:
    """Creates a Daft Catalog backed by the AWS Glue service, with optional client or session.

    Terms:
        - AWS Glue          -> Daft Catalog
        - AWS Glue Database -> Daft Namespace
        - AWS Glue Table    -> Daft Table

    Args:
        name (str): glue database name
        client: optional boto3 client
        session: optional boto3 session

    Returns:
        Catalog: new daft catalog instance backed by AWS Glue.
    """
    try:
        from daft.catalog.__glue import GlueCatalog

        if client is not None and session is not None:
            raise ValueError("Can provide either a client or session but not both.")
        elif client is not None:
            return GlueCatalog.from_client(name, client)
        elif session is not None:
            return GlueCatalog.from_session(name, session)
        else:
            raise ValueError("Must provide either a client or session.")
    except ImportError:
        raise ImportError("AWS Glue support not installed: pip install -U 'daft[aws]'")

from_iceberg #

from_iceberg(catalog: object) -> Catalog

Create a Daft Catalog from a PyIceberg catalog object.

Parameters:

Name Type Description Default
catalog object

a PyIceberg catalog instance

required

Returns:

Name Type Description
Catalog Catalog

a new Catalog instance backed by the PyIceberg catalog.

Examples:

1
2
3
>>> from pyiceberg.catalog import load_catalog
>>> iceberg_catalog = load_catalog("my_iceberg_catalog")
>>> catalog = Catalog.from_iceberg(iceberg_catalog)
Source code in daft/catalog/__init__.py
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
@staticmethod
def from_iceberg(catalog: object) -> Catalog:
    """Create a Daft Catalog from a PyIceberg catalog object.

    Args:
        catalog (object): a PyIceberg catalog instance

    Returns:
        Catalog: a new Catalog instance backed by the PyIceberg catalog.

    Examples:
        >>> from pyiceberg.catalog import load_catalog
        >>> iceberg_catalog = load_catalog("my_iceberg_catalog")
        >>> catalog = Catalog.from_iceberg(iceberg_catalog)
    """
    try:
        from daft.catalog.__iceberg import IcebergCatalog

        return IcebergCatalog._from_obj(catalog)
    except ImportError:
        raise ImportError("Iceberg support not installed: pip install -U 'daft[iceberg]'")

from_pydict #

from_pydict(
    tables: dict[Identifier | str, object],
    name: str = "default",
) -> Catalog

Returns an in-memory catalog from a dictionary of table-like objects.

The table-like objects can be pydicts, dataframes, or a Table implementation. For qualified tables, namespaces are created if necessary.

Parameters:

Name Type Description Default
tables dict[str, object]

a dictionary of table-like objects (pydicts, dataframes, and tables)

required

Returns:

Name Type Description
Catalog Catalog

new catalog instance with name 'default'

Examples:

 1
 2
 3
 4
 5
 6
 7
 8
 9
10
11
12
13
14
15
16
>>> import daft
>>> from daft.catalog import Catalog, Table
>>>
>>> dictionary = {"x": [1, 2, 3]}
>>> dataframe = daft.from_pydict(dictionary)
>>> table = Table.from_df("temp", dataframe)
>>>
>>> catalog = Catalog.from_pydict(
...     {
...         "R": dictionary,
...         "S": dataframe,
...         "T": table,
...     }
... )
>>> catalog.list_tables()
['R', 'S', 'T']
Source code in daft/catalog/__init__.py
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
@staticmethod
def from_pydict(tables: dict[Identifier | str, object], name: str = "default") -> Catalog:
    """Returns an in-memory catalog from a dictionary of table-like objects.

    The table-like objects can be pydicts, dataframes, or a Table implementation.
    For qualified tables, namespaces are created if necessary.

    Args:
        tables (dict[str,object]): a dictionary of table-like objects (pydicts, dataframes, and tables)

    Returns:
        Catalog: new catalog instance with name 'default'

    Examples:
        >>> import daft
        >>> from daft.catalog import Catalog, Table
        >>>
        >>> dictionary = {"x": [1, 2, 3]}
        >>> dataframe = daft.from_pydict(dictionary)
        >>> table = Table.from_df("temp", dataframe)
        >>>
        >>> catalog = Catalog.from_pydict(
        ...     {
        ...         "R": dictionary,
        ...         "S": dataframe,
        ...         "T": table,
        ...     }
        ... )
        >>> catalog.list_tables()
        ['R', 'S', 'T']

    """
    from daft.catalog.__internal import MemoryCatalog

    catalog = MemoryCatalog._new(name)

    for ident, source in tables.items():
        ident = ident if isinstance(ident, Identifier) else Identifier.from_str(ident)

        # has namespace, create one if it doesn't exist
        if len(ident) > 1:
            namespace = Identifier(*ident[:-1])
            catalog.create_namespace_if_not_exists(namespace)

        df: DataFrame
        if isinstance(source, Table):
            df = source.read()
        elif isinstance(source, DataFrame):
            df = source
        elif isinstance(source, dict):
            df = DataFrame._from_pydict(source)
        else:
            raise ValueError(f"Unsupported table source {type(source)}")

        catalog.create_table(ident, df)

    return catalog

from_s3tables #

from_s3tables(
    table_bucket_arn: str,
    client: object | None = None,
    session: object | None = None,
) -> Catalog

Creates a Daft Catalog from S3 Tables bucket ARN, with optional client or session.

If neither a boto3 client nor session is provided, the Iceberg REST client will be used under the hood.

Parameters:

Name Type Description Default
table_bucket_arn str

ARN of the S3 Tables bucket

required
client object

a boto3 client

None
session object

a boto3 session

None

Returns:

Name Type Description
Catalog Catalog

a new Catalog instance backed by S3 Tables.

Examples:

1
2
3
>>> arn = "arn:aws:s3:::my-s3tables-bucket"
>>> catalog = Catalog.from_s3tables(arn)
>>> catalog.list_tables()
Source code in daft/catalog/__init__.py
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
@staticmethod
def from_s3tables(
    table_bucket_arn: str,
    client: object | None = None,
    session: object | None = None,
) -> Catalog:
    """Creates a Daft Catalog from S3 Tables bucket ARN, with optional client or session.

    If neither a boto3 client nor session is provided, the Iceberg REST
    client will be used under the hood.

    Args:
        table_bucket_arn (str): ARN of the S3 Tables bucket
        client (object, optional): a boto3 client
        session (object, optional): a boto3 session

    Returns:
        Catalog: a new Catalog instance backed by S3 Tables.

    Examples:
        >>> arn = "arn:aws:s3:::my-s3tables-bucket"
        >>> catalog = Catalog.from_s3tables(arn)
        >>> catalog.list_tables()
    """
    try:
        from daft.catalog.__s3tables import S3Catalog

        if client is not None and session is not None:
            raise ValueError("Can provide either a client or session but not both.")
        elif client is not None:
            return S3Catalog.from_client(table_bucket_arn, client)
        elif session is not None:
            return S3Catalog.from_session(table_bucket_arn, session)
        else:
            return S3Catalog.from_arn(table_bucket_arn)
    except ImportError:
        raise ImportError("S3 Tables support not installed: pip install -U 'daft[aws]'")

from_unity #

from_unity(catalog: object) -> Catalog

Create a Daft Catalog from a Unity Catalog client.

Parameters:

Name Type Description Default
catalog object

a Unity Catalog client instance

required

Returns:

Name Type Description
Catalog Catalog

a new Catalog instance backed by the Unity catalog.

Examples:

1
2
3
>>> from unity_sdk import UnityCatalogClient
>>> unity_client = UnityCatalogClient(...)
>>> catalog = Catalog.from_unity(unity_client)
Source code in daft/catalog/__init__.py
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
@staticmethod
def from_unity(catalog: object) -> Catalog:
    """Create a Daft Catalog from a Unity Catalog client.

    Args:
        catalog (object): a Unity Catalog client instance

    Returns:
        Catalog: a new Catalog instance backed by the Unity catalog.

    Examples:
        >>> from unity_sdk import UnityCatalogClient
        >>> unity_client = UnityCatalogClient(...)
        >>> catalog = Catalog.from_unity(unity_client)

    """
    try:
        from daft.catalog.__unity import UnityCatalog

        return UnityCatalog._from_obj(catalog)
    except ImportError:
        raise ImportError("Unity support not installed: pip install -U 'daft[unity]'")

get_table #

get_table(identifier: Identifier | str) -> Table

Get a table by its identifier or raises if the table does not exist.

Parameters:

Name Type Description Default
identifier Identifier | str

table identifier

required

Returns:

Name Type Description
Table Table

matched table or raises if the table does not exist.

Source code in daft/catalog/__init__.py
429
430
431
432
433
434
435
436
437
438
439
440
441
def get_table(self, identifier: Identifier | str) -> Table:
    """Get a table by its identifier or raises if the table does not exist.

    Args:
        identifier (Identifier|str): table identifier

    Returns:
        Table: matched table or raises if the table does not exist.
    """
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)

    return self._get_table(identifier)

has_namespace #

has_namespace(identifier: Identifier | str) -> bool

Returns True if the namespace exists, otherwise False.

Source code in daft/catalog/__init__.py
395
396
397
398
399
400
def has_namespace(self, identifier: Identifier | str) -> bool:
    """Returns True if the namespace exists, otherwise False."""
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)

    return self._has_namespace(identifier)

has_table #

has_table(identifier: Identifier | str) -> bool

Returns True if the table exists, otherwise False.

Source code in daft/catalog/__init__.py
402
403
404
405
406
407
def has_table(self, identifier: Identifier | str) -> bool:
    """Returns True if the table exists, otherwise False."""
    if isinstance(identifier, str):
        identifier = Identifier.from_str(identifier)

    return self._has_table(identifier)

list_namespaces #

list_namespaces(
    pattern: str | None = None,
) -> list[Identifier]

List namespaces in the catalog which match the given pattern.

Parameters:

Name Type Description Default
pattern str

pattern to match such as a namespace prefix

None

Returns:

Type Description
list[Identifier]

list[Identifier]: list of namespace identifiers matching the pattern.

Source code in daft/catalog/__init__.py
447
448
449
450
451
452
453
454
455
456
def list_namespaces(self, pattern: str | None = None) -> list[Identifier]:
    """List namespaces in the catalog which match the given pattern.

    Args:
        pattern (str): pattern to match such as a namespace prefix

    Returns:
        list[Identifier]: list of namespace identifiers matching the pattern.
    """
    return self._list_namespaces(pattern)

list_tables #

list_tables(pattern: str | None = None) -> list[Identifier]

List tables in the catalog which match the given pattern.

Parameters:

Name Type Description Default
pattern str

pattern to match such as a namespace prefix

None

Returns:

Type Description
list[Identifier]

list[str]: list of table identifiers matching the pattern.

Source code in daft/catalog/__init__.py
458
459
460
461
462
463
464
465
466
467
def list_tables(self, pattern: str | None = None) -> list[Identifier]:
    """List tables in the catalog which match the given pattern.

    Args:
        pattern (str): pattern to match such as a namespace prefix

    Returns:
        list[str]: list of table identifiers matching the pattern.
    """
    return self._list_tables(pattern)

read_table #

read_table(
    identifier: Identifier | str, **options: dict[str, Any]
) -> DataFrame

Returns the table as a DataFrame or raises an exception if it does not exist.

Source code in daft/catalog/__init__.py
473
474
475
def read_table(self, identifier: Identifier | str, **options: dict[str, Any]) -> DataFrame:
    """Returns the table as a DataFrame or raises an exception if it does not exist."""
    return self.get_table(identifier).read(**options)

write_table #

write_table(
    identifier: Identifier | str,
    df: DataFrame,
    mode: Literal["append", "overwrite"] = "append",
    **options: dict[str, Any],
) -> None
Source code in daft/catalog/__init__.py
481
482
483
484
485
486
487
488
def write_table(
    self,
    identifier: Identifier | str,
    df: DataFrame,
    mode: Literal["append", "overwrite"] = "append",
    **options: dict[str, Any],
) -> None:
    return self.get_table(identifier).write(df, mode=mode, **options)

Identifier #

Identifier(*parts: str)

A reference (path) to a catalog object.

Examples:

1
2
>>> id = Identifier("a", "b")
>>> assert len(id) == 2

Creates an Identifier from its parts.

Examples:

1
2
>>> from daft.catalog import Identifier
>>> Identifier("namespace", "table")

Methods:

Name Description
drop

Returns a new Identifier with the first n parts removed.

from_sql

Parses an Identifier from an SQL string, normalizing to lowercase if specified.

from_str

Parses an Identifier from a dot-delimited Python string without normalization.

Source code in daft/catalog/__init__.py
508
509
510
511
512
513
514
515
516
517
def __init__(self, *parts: str):
    """Creates an Identifier from its parts.

    Examples:
        >>> from daft.catalog import Identifier
        >>> Identifier("namespace", "table")
    """
    if len(parts) < 1:
        raise ValueError("Identifier requires at least one part.")
    self._ident = PyIdentifier(parts)

drop #

drop(n: int = 1) -> Identifier

Returns a new Identifier with the first n parts removed.

Parameters:

Name Type Description Default
n int

Number of parts to drop from the beginning. Defaults to 1.

1

Returns:

Name Type Description
Identifier Identifier

A new Identifier with the first n parts removed.

Raises:

Type Description
ValueError

If dropping n parts would result in an empty Identifier.

Source code in daft/catalog/__init__.py
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
def drop(self, n: int = 1) -> Identifier:
    """Returns a new Identifier with the first n parts removed.

    Args:
        n (int): Number of parts to drop from the beginning. Defaults to 1.

    Returns:
        Identifier: A new Identifier with the first n parts removed.

    Raises:
        ValueError: If dropping n parts would result in an empty Identifier.
    """
    if n <= 0:
        return Identifier(*self)
    if n >= len(self):
        raise ValueError(f"Cannot drop {n} parts from Identifier with {len(self)} parts")
    parts = tuple(self)
    return Identifier(*parts[n:])

from_sql #

from_sql(input: str, normalize: bool = False) -> Identifier

Parses an Identifier from an SQL string, normalizing to lowercase if specified.

Parameters:

Name Type Description Default
input str

input sql string

required
normalize bool

flag to case-normalize the identifier text

False

Returns:

Name Type Description
Identifier Identifier

new identifier instance

Examples:

1
2
3
4
>>> from daft.catalog import Identifier
>>> Identifier.from_sql("namespace.table") == Identifier("namespace", "table")
>>> Identifier.from_sql('"a.b"') == Identifier('"a.b."')
>>> Identifier.from_sql('ABC."xYz"', normalize=True) == Identifier("abc", "xYz")
Source code in daft/catalog/__init__.py
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
@staticmethod
def from_sql(input: str, normalize: bool = False) -> Identifier:
    """Parses an Identifier from an SQL string, normalizing to lowercase if specified.

    Args:
        input (str): input sql string
        normalize (bool): flag to case-normalize the identifier text

    Returns:
        Identifier: new identifier instance

    Examples:
        >>> from daft.catalog import Identifier
        >>> Identifier.from_sql("namespace.table") == Identifier("namespace", "table")
        >>> Identifier.from_sql('"a.b"') == Identifier('"a.b."')
        >>> Identifier.from_sql('ABC."xYz"', normalize=True) == Identifier("abc", "xYz")

    """
    i = Identifier.__new__(Identifier)
    i._ident = PyIdentifier.from_sql(input, normalize)
    return i

from_str #

from_str(input: str) -> Identifier

Parses an Identifier from a dot-delimited Python string without normalization.

Parameters:

Name Type Description Default
input str

input identifier string

required

Returns:

Name Type Description
Identifier Identifier

new identifier instance

Examples:

1
2
>>> from daft.catalog import Identifier
>>> Identifier.from_str("namespace.table") == Identifier("namespace", "table")
Source code in daft/catalog/__init__.py
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
@staticmethod
def from_str(input: str) -> Identifier:
    """Parses an Identifier from a dot-delimited Python string without normalization.

    Args:
        input (str): input identifier string

    Returns:
        Identifier: new identifier instance

    Examples:
        >>> from daft.catalog import Identifier
        >>> Identifier.from_str("namespace.table") == Identifier("namespace", "table")

    """
    return Identifier(*input.split("."))

Table #

Interface for python table implementations.

Methods:

Name Description
append

Appends the DataFrame to this table.

from_df

Returns a read-only table backed by the DataFrame.

from_iceberg

Creates a Daft Table instance from an Iceberg table.

from_pydict

Returns a read-only table backed by the given data.

from_unity

Returns a Daft Table instance from a Unity table.

overwrite

Overwrites this table with the given DataFrame.

read

Creates a new DataFrame from this table.

schema

Returns the table's schema.

select

Creates a new DataFrame from the table applying the provided expressions.

show

Shows the first n rows from this table.

write

Writes the DataFrame to this table.

Attributes:

Name Type Description
name str

Returns the table's name.

name #

name: str

Returns the table's name.

append #

append(df: DataFrame, **options: Any) -> None

Appends the DataFrame to this table.

Parameters:

Name Type Description Default
df DataFrame

dataframe to append

required
**options Any

additional format-dependent write options

{}
Source code in daft/catalog/__init__.py
792
793
794
795
796
797
798
799
@abstractmethod
def append(self, df: DataFrame, **options: Any) -> None:
    """Appends the DataFrame to this table.

    Args:
        df (DataFrame): dataframe to append
        **options (Any): additional format-dependent write options
    """

from_df #

from_df(name: str, dataframe: DataFrame) -> Table

Returns a read-only table backed by the DataFrame.

Parameters:

Name Type Description Default
name str

table name

required
dataframe DataFrame

table source dataframe

required

Returns:

Name Type Description
Table Table

new table instance

Examples:

1
2
3
>>> import daft
>>> from daft.catalog import Table
>>> Table.from_df("my_table", daft.from_pydict({"x": [1, 2, 3]}))
Source code in daft/catalog/__init__.py
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
@staticmethod
def from_df(name: str, dataframe: DataFrame) -> Table:
    """Returns a read-only table backed by the DataFrame.

    Args:
        name (str): table name
        dataframe (DataFrame): table source dataframe

    Returns:
        Table: new table instance

    Examples:
        >>> import daft
        >>> from daft.catalog import Table
        >>> Table.from_df("my_table", daft.from_pydict({"x": [1, 2, 3]}))

    """
    from daft.catalog.__internal import MemoryTable

    table = MemoryTable._new(name, dataframe.schema())
    table.append(dataframe)

    return table

from_iceberg #

from_iceberg(table: object) -> Table

Creates a Daft Table instance from an Iceberg table.

Parameters:

Name Type Description Default
table object

a pyiceberg table

required

Returns:

Name Type Description
Table Table

new daft table instance

Source code in daft/catalog/__init__.py
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
@staticmethod
def from_iceberg(table: object) -> Table:
    """Creates a Daft Table instance from an Iceberg table.

    Args:
        table (object): a pyiceberg table

    Returns:
        Table: new daft table instance
    """
    try:
        from daft.catalog.__iceberg import IcebergTable

        return IcebergTable._from_obj(table)
    except ImportError:
        raise ImportError("Iceberg support not installed: pip install -U 'daft[iceberg]'")

from_pydict #

from_pydict(
    name: str, data: dict[str, InputListType]
) -> Table

Returns a read-only table backed by the given data.

Parameters:

Name Type Description Default
name str

table table

required
data dict[str, object]

keys are column names and the values are python lists, numpy arrays, or arrow arrays.

required

Returns:

Name Type Description
DataFrame Table

new read-only table instance

Examples:

 1
 2
 3
 4
 5
 6
 7
 8
 9
10
11
12
13
14
>>> from daft.catalog import Table
>>> table = Table.from_pydict({"foo": [1, 2]})
>>> table.show()
╭───────╮
│ foo   │
│ ---   │
│ Int64 │
╞═══════╡
│ 1     │
├╌╌╌╌╌╌╌┤
│ 2     │
╰───────╯

(Showing first 2 of 2 rows)
Source code in daft/catalog/__init__.py
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
@staticmethod
def from_pydict(name: str, data: dict[str, InputListType]) -> Table:
    """Returns a read-only table backed by the given data.

    Args:
        name (str): table table
        data dict[str,object]: keys are column names and the values are python lists, numpy arrays, or arrow arrays.

    Returns:
        DataFrame: new read-only table instance

    Examples:
        >>> from daft.catalog import Table
        >>> table = Table.from_pydict({"foo": [1, 2]})
        >>> table.show()
        ╭───────╮
        │ foo   │
        │ ---   │
        │ Int64 │
        ╞═══════╡
        │ 1     │
        ├╌╌╌╌╌╌╌┤
        │ 2     │
        ╰───────╯
        <BLANKLINE>
        (Showing first 2 of 2 rows)

    """
    df = DataFrame._from_pydict(data)
    return Table.from_df(name, df)

from_unity #

from_unity(table: object) -> Table

Returns a Daft Table instance from a Unity table.

Parameters:

Name Type Description Default
table object

unity table instance.

required
Source code in daft/catalog/__init__.py
697
698
699
700
701
702
703
704
705
706
707
708
709
@staticmethod
def from_unity(table: object) -> Table:
    """Returns a Daft Table instance from a Unity table.

    Args:
        table (object): unity table instance.
    """
    try:
        from daft.catalog.__unity import UnityTable

        return UnityTable._from_obj(table)
    except ImportError:
        raise ImportError("Unity support not installed: pip install -U 'daft[unity]'")

overwrite #

overwrite(df: DataFrame, **options: Any) -> None

Overwrites this table with the given DataFrame.

Parameters:

Name Type Description Default
df DataFrame

dataframe to overwrite this table with

required
**options Any

additional format-dependent write options

{}
Source code in daft/catalog/__init__.py
801
802
803
804
805
806
807
808
@abstractmethod
def overwrite(self, df: DataFrame, **options: Any) -> None:
    """Overwrites this table with the given DataFrame.

    Args:
        df (DataFrame): dataframe to overwrite this table with
        **options (Any): additional format-dependent write options
    """

read #

read(**options: Any) -> DataFrame

Creates a new DataFrame from this table.

Parameters:

Name Type Description Default
**options Any

additional format-dependent read options

{}

Returns:

Name Type Description
DataFrame DataFrame

new DataFrame instance

Source code in daft/catalog/__init__.py
744
745
746
747
748
749
750
751
752
753
@abstractmethod
def read(self, **options: Any) -> DataFrame:
    """Creates a new DataFrame from this table.

    Args:
        **options (Any): additional format-dependent read options

    Returns:
        DataFrame: new DataFrame instance
    """

schema #

schema() -> Schema

Returns the table's schema.

Source code in daft/catalog/__init__.py
621
622
623
@abstractmethod
def schema(self) -> Schema:
    """Returns the table's schema."""

select #

select(*columns: ColumnInputType) -> DataFrame

Creates a new DataFrame from the table applying the provided expressions.

Parameters:

Name Type Description Default
*columns Expression | str

columns to select from the current DataFrame

()

Returns:

Name Type Description
DataFrame DataFrame

new DataFrame instance with the select columns

Source code in daft/catalog/__init__.py
755
756
757
758
759
760
761
762
763
764
def select(self, *columns: ColumnInputType) -> DataFrame:
    """Creates a new DataFrame from the table applying the provided expressions.

    Args:
        *columns (Expression|str): columns to select from the current DataFrame

    Returns:
        DataFrame: new DataFrame instance with the select columns
    """
    return self.read().select(*columns)

show #

show(n: int = 8) -> None

Shows the first n rows from this table.

Parameters:

Name Type Description Default
n int

number of rows to show

8

Returns:

Type Description
None

None

Source code in daft/catalog/__init__.py
766
767
768
769
770
771
772
773
774
775
def show(self, n: int = 8) -> None:
    """Shows the first n rows from this table.

    Args:
        n (int): number of rows to show

    Returns:
        None
    """
    return self.read().show(n)

write #

write(
    df: DataFrame,
    mode: Literal["append", "overwrite"] = "append",
    **options: Any,
) -> None

Writes the DataFrame to this table.

Parameters:

Name Type Description Default
df DataFrame

datafram to write

required
mode str

write mode such as 'append' or 'overwrite'

'append'
**options Any

additional format-dependent write options

{}
Source code in daft/catalog/__init__.py
781
782
783
784
785
786
787
788
789
790
def write(self, df: DataFrame, mode: Literal["append", "overwrite"] = "append", **options: Any) -> None:
    """Writes the DataFrame to this table.

    Args:
        df (DataFrame): datafram to write
        mode (str): write mode such as 'append' or 'overwrite'
        **options (Any): additional format-dependent write options
    """
    if mode == "append":
        return self.append(df, **options)