PyIceberg
Much of the python api conforms to the Java API. You can get more info about the java api here.
Installing
You can install the latest release version from pypi:
pip3 install "pyiceberg[s3fs,hive]"
Or install the latest development version locally:
pip3 install poetry --upgrade
pip3 install -e ".[s3fs,hive]"
With optional dependencies:
Key | Description: |
---|---|
hive | Support for the Hive metastore |
pyarrow | PyArrow as a FileIO implementation to interact with the object store |
s3fs | S3FS as a FileIO implementation to interact with the object store |
snappy | Support for snappy Avro compression |
Python CLI Quickstart
Pyiceberg ships with a CLI that's available after installing the pyiceberg
package.
➜ pyiceberg --help
Usage: pyiceberg [OPTIONS] COMMAND [ARGS]...
Options:
--catalog TEXT
--verbose BOOLEAN
--output [text|json]
--uri TEXT
--credential TEXT
--help Show this message and exit.
Commands:
describe Describes a namespace xor table
drop Operations to drop a namespace or table
list Lists tables or namespaces
location Returns the location of the table
properties Properties on tables/namespaces
rename Renames a table
schema Gets the schema of the table
spec Returns the partition spec of the table
uuid Returns the UUID of the table
Configuration
There are three ways of setting the configuration.
For the CLI you can pass it in using --uri
and --credential
and it will automatically detect the type based on the scheme (http(s)
for rest, thrift
for Hive).
Secondly, YAML based configuration is supported cat ~/.pyiceberg.yaml
:
catalog:
default:
uri: thrift://localhost:9083
rest:
uri: http://rest-catalog/ws/
credential: t-1234:secret
Lastly, you can also set it using environment variables:
export PYICEBERG_CATALOG__DEFAULT__URI=thrift://localhost:9083
export PYICEBERG_CATALOG__REST__URI=http://rest-catalog/ws/
export PYICEBERG_CATALOG__REST__CREDENTIAL=t-1234:secret
Where the structure is equivalent to the YAML. The levels are separated using a double underscore (__
).
CLI Quickstart
This example assumes that you have a default catalog set. If you want to load another catalog, for example, the rest example above. Then you need to set --catalog rest
.
➜ pyiceberg list
default
nyc
➜ pyiceberg list nyc
nyc.taxis
pyiceberg describe nyc.taxis
Table format version 1
Metadata location file:/.../nyc.db/taxis/metadata/00000-aa3a3eac-ea08-4255-b890-383a64a94e42.metadata.json
Table UUID 6cdfda33-bfa3-48a7-a09e-7abb462e3460
Last Updated 1661783158061
Partition spec []
Sort order []
Current schema Schema, id=0
├── 1: VendorID: optional long
├── 2: tpep_pickup_datetime: optional timestamptz
├── 3: tpep_dropoff_datetime: optional timestamptz
├── 4: passenger_count: optional double
├── 5: trip_distance: optional double
├── 6: RatecodeID: optional double
├── 7: store_and_fwd_flag: optional string
├── 8: PULocationID: optional long
├── 9: DOLocationID: optional long
├── 10: payment_type: optional long
├── 11: fare_amount: optional double
├── 12: extra: optional double
├── 13: mta_tax: optional double
├── 14: tip_amount: optional double
├── 15: tolls_amount: optional double
├── 16: improvement_surcharge: optional double
├── 17: total_amount: optional double
├── 18: congestion_surcharge: optional double
└── 19: airport_fee: optional double
Current snapshot Operation.APPEND: id=5937117119577207079, schema_id=0
Snapshots Snapshots
└── Snapshot 5937117119577207079, schema 0: file:/.../nyc.db/taxis/metadata/snap-5937117119577207079-1-94656c4f-4c66-4600-a4ca-f30377300527.avro
Properties owner root
write.format.default parquet
Or output in JSON for automation:
pyiceberg --output json describe nyc.taxis | jq
{
"identifier": [
"nyc",
"taxis"
],
"metadata_location": "file:/.../nyc.db/taxis/metadata/00000-aa3a3eac-ea08-4255-b890-383a64a94e42.metadata.json",
"metadata": {
"location": "file:/.../nyc.db/taxis",
"table-uuid": "6cdfda33-bfa3-48a7-a09e-7abb462e3460",
"last-updated-ms": 1661783158061,
"last-column-id": 19,
"schemas": [
{
"type": "struct",
"fields": [
{
"id": 1,
"name": "VendorID",
"type": "long",
"required": false
},
...
{
"id": 19,
"name": "airport_fee",
"type": "double",
"required": false
}
],
"schema-id": 0,
"identifier-field-ids": []
}
],
"current-schema-id": 0,
"partition-specs": [
{
"spec-id": 0,
"fields": []
}
],
"default-spec-id": 0,
"last-partition-id": 999,
"properties": {
"owner": "root",
"write.format.default": "parquet"
},
"current-snapshot-id": 5937117119577207000,
"snapshots": [
{
"snapshot-id": 5937117119577207000,
"timestamp-ms": 1661783158061,
"manifest-list": "file:/.../nyc.db/taxis/metadata/snap-5937117119577207079-1-94656c4f-4c66-4600-a4ca-f30377300527.avro",
"summary": {
"operation": "append",
"spark.app.id": "local-1661783139151",
"added-data-files": "1",
"added-records": "2979431",
"added-files-size": "46600777",
"changed-partition-count": "1",
"total-records": "2979431",
"total-files-size": "46600777",
"total-data-files": "1",
"total-delete-files": "0",
"total-position-deletes": "0",
"total-equality-deletes": "0"
},
"schema-id": 0
}
],
"snapshot-log": [
{
"snapshot-id": "5937117119577207079",
"timestamp-ms": 1661783158061
}
],
"metadata-log": [],
"sort-orders": [
{
"order-id": 0,
"fields": []
}
],
"default-sort-order-id": 0,
"refs": {
"main": {
"snapshot-id": 5937117119577207000,
"type": "branch"
}
},
"format-version": 1,
"schema": {
"type": "struct",
"fields": [
{
"id": 1,
"name": "VendorID",
"type": "long",
"required": false
},
...
{
"id": 19,
"name": "airport_fee",
"type": "double",
"required": false
}
],
"schema-id": 0,
"identifier-field-ids": []
},
"partition-spec": []
}
}
Python API
To instantiate a catalog:
>>> from pyiceberg.catalog.hive import HiveCatalog
>>> catalog = HiveCatalog(name='prod', uri='thrift://localhost:9083/')
>>> catalog.list_namespaces()
[('default',), ('nyc',)]
>>> catalog.list_tables('nyc')
[('nyc', 'taxis')]
>>> catalog.load_table(('nyc', 'taxis'))
Table(identifier=('nyc', 'taxis'), ...)
And to create a table from a catalog:
from pyiceberg.schema import Schema
from pyiceberg.types import TimestampType, DoubleType, StringType, NestedField
schema = Schema(
NestedField(field_id=1, name="datetime", field_type=TimestampType(), required=False),
NestedField(field_id=2, name="bid", field_type=DoubleType(), required=False),
NestedField(field_id=3, name="ask", field_type=DoubleType(), required=False),
NestedField(field_id=4, name="symbol", field_type=StringType(), required=False),
)
from pyiceberg.table.partitioning import PartitionSpec, PartitionField
from pyiceberg.transforms import DayTransform
partition_spec = PartitionSpec(
PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name="datetime_day")
)
from pyiceberg.table.sorting import SortOrder, SortField
from pyiceberg.transforms import IdentityTransform
sort_order = SortOrder(
SortField(source_id=4, transform=IdentityTransform())
)
from pyiceberg.catalog.hive import HiveCatalog
catalog = HiveCatalog(name='prod', uri='thrift://localhost:9083/')
catalog.create_table(
identifier='default.bids',
location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/',
schema=schema,
partition_spec=partition_spec,
sort_order=sort_order
)
Which returns a newly created table:
Table(
identifier=('default', 'bids'),
metadata_location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids//metadata/00000-c8cd93ab-f784-474d-a167-b1a86b05195f.metadata.json',
metadata=TableMetadataV2(
location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/',
table_uuid=UUID('38d4cb39-4945-4bf2-b374-984b5c4984d2'),
last_updated_ms=1661847562069,
last_column_id=4,
schemas=[
Schema(
NestedField(field_id=1, name='datetime', field_type=TimestampType(), required=False),
NestedField(field_id=2, name='bid', field_type=DoubleType(), required=False),
NestedField(field_id=3, name='ask', field_type=DoubleType(), required=False),
NestedField(field_id=4, name='symbol', field_type=StringType(), required=False)),
schema_id=1,
identifier_field_ids=[])
],
current_schema_id=1,
partition_specs=[
PartitionSpec(
PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name='datetime_day'),))
],
default_spec_id=0,
last_partition_id=1000,
properties={},
current_snapshot_id=None,
snapshots=[],
snapshot_log=[],
metadata_log=[],
sort_orders=[
SortOrder(order_id=1, fields=[SortField(source_id=4, transform=IdentityTransform(), direction=SortDirection.ASC, null_order=NullOrder.NULLS_FIRST)])
],
default_sort_order_id=1,
refs={},
format_version=2,
last_sequence_number=0
)
)
Feature Support
The goal is that the python library will provide a functional, performant subset of the Java library. The initial focus has been on reading table metadata and provide a convenient CLI to go through the catalog.
Metadata
Operation | Java | Python |
---|---|---|
Get Schema | X | X |
Get Snapshots | X | X |
Plan Scan | X | |
Plan Scan for Snapshot | X | |
Update Current Snapshot | X | |
Set Table Properties | X | X |
Create Table | X | X |
Drop Table | X | X |
Alter Table | X |
Types
The types are located in pyiceberg.types
.
Primitive types:
- BooleanType
- StringType
- IntegerType
- LongType
- FloatType
- DoubleType
- DateType
- TimeType
- TimestampType
- TimestamptzType
- BinaryType
- UUIDType
Complex types:
- StructType
- ListType
- MapType
- FixedType(16)
- DecimalType(8, 3)