This is an automated email from the ASF dual-hosted git repository.

fokko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 0387308bfc Python: Improve docs (#6389)
0387308bfc is described below

commit 0387308bfc291645d3ad8134a101cf5c7e4ee3d2
Author: Fokko Driesprong <[email protected]>
AuthorDate: Fri Dec 9 13:34:50 2022 +0100

    Python: Improve docs (#6389)
    
    * Python: Improve docs
    
    This improves the docs by turning into a multi page site.
    
    Also merges in the `CONTRIBUTING.md` and `RELEASE.md` to have
    everything into a single place.
    
    Also changed to the readthedocs theme since that has a much
    cleaner multi page layout (navigation on the left instead
    of the top).
    
    * Rework the docs
---
 python/.pre-commit-config.yaml                     |  18 +-
 python/mkdocs/docs/api.md                          | 339 +++++++++++++
 python/mkdocs/docs/cli.md                          | 215 ++++++++
 python/mkdocs/docs/configuration.md                |  84 ++++
 python/mkdocs/docs/contributing.md                 | 135 +++++
 python/mkdocs/docs/feature-support.md              |  85 ++++
 .../RELEASE.md => mkdocs/docs/how-to-release.md}   |  22 +-
 python/mkdocs/docs/index.md                        | 550 +--------------------
 python/mkdocs/docs/verify-release.md               | 100 ++++
 python/mkdocs/mkdocs.yml                           |  16 +
 10 files changed, 1019 insertions(+), 545 deletions(-)

diff --git a/python/.pre-commit-config.yaml b/python/.pre-commit-config.yaml
index e6f71b94de..1c367e5644 100644
--- a/python/.pre-commit-config.yaml
+++ b/python/.pre-commit-config.yaml
@@ -63,11 +63,13 @@ repos:
       - id: flake8
         args: [ "--ignore=E501,W503,E203,B024" ]
         additional_dependencies: [ flake8-bugbear==22.9.11, 
flake8-comprehensions==3.10.0 ]
-  - repo: https://github.com/executablebooks/mdformat
-    rev: 0.7.16
-    hooks:
-      - id: mdformat
-        additional_dependencies:
-          - mdformat-black
-          - mdformat-config
-          - mdformat-beautysh
+# Disabling this for now, we need mdformat-admin, otherwise it will break the 
note blocks
+#  - repo: https://github.com/executablebooks/mdformat
+#    rev: 0.7.16
+#    hooks:
+#      - id: mdformat
+#        additional_dependencies:
+#          - mdformat-black
+#          - mdformat-config
+#          - mdformat-beautysh
+#          - mdformat-admon
diff --git a/python/mkdocs/docs/api.md b/python/mkdocs/docs/api.md
new file mode 100644
index 0000000000..dd6423f61a
--- /dev/null
+++ b/python/mkdocs/docs/api.md
@@ -0,0 +1,339 @@
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -   http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing,
+  - software distributed under the License is distributed on an
+  - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  - KIND, either express or implied.  See the License for the
+  - specific language governing permissions and limitations
+  - under the License.
+  -->
+
+# Python API
+
+PyIceberg is based around catalogs to load tables. First step is to 
instantiate a catalog that loads tables. Let's use the following configuration:
+
+```yaml
+catalog:
+  prod:
+    uri: http://rest-catalog/ws/
+    credential: t-1234:secret
+```
+
+Then load the `prod` catalog:
+
+```python
+from pyiceberg.catalog import load_catalog
+
+catalog = load_catalog("prod")
+
+catalog.list_namespaces()
+```
+
+Returns two namespaces:
+
+```python
+[("default",), ("nyc",)]
+```
+
+Listing the tables in the `nyc` namespace:
+
+```python
+catalog.list_tables("nyc")
+```
+
+Returns as list with tuples, containing a single table `taxis`:
+
+```python
+[("nyc", "taxis")]
+```
+
+## Load a table
+
+Loading the `taxis` table:
+
+```python
+catalog.load_table("nyc.taxis")
+# Equivalent to:
+catalog.load_table(("nyc", "taxis"))
+# The tuple syntax can be used if the namespace or table contains a dot.
+```
+
+This returns a `Table` that represents an Iceberg table:
+
+```python
+Table(
+  identifier=('nyc', 'taxis'),
+  
metadata_location='s3a://warehouse/wh/nyc.db/taxis/metadata/00002-6ea51ce3-62aa-4197-9cf8-43d07c3440ca.metadata.json',
+  metadata=TableMetadataV2(
+    location='s3a://warehouse/wh/nyc.db/taxis',
+    table_uuid=UUID('ebd5d172-2162-453d-b586-1cdce52c1116'),
+    last_updated_ms=1662633437826,
+    last_column_id=19,
+    schemas=[Schema(
+        NestedField(field_id=1, name='VendorID', field_type=LongType(), 
required=False),
+        NestedField(field_id=2, name='tpep_pickup_datetime', 
field_type=TimestamptzType(), required=False),
+        NestedField(field_id=3, name='tpep_dropoff_datetime', 
field_type=TimestamptzType(), required=False),
+        NestedField(field_id=4, name='passenger_count', 
field_type=DoubleType(), required=False),
+        NestedField(field_id=5, name='trip_distance', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=6, name='RatecodeID', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=7, name='store_and_fwd_flag', 
field_type=StringType(), required=False),
+        NestedField(field_id=8, name='PULocationID', field_type=LongType(), 
required=False),
+        NestedField(field_id=9, name='DOLocationID', field_type=LongType(), 
required=False),
+        NestedField(field_id=10, name='payment_type', field_type=LongType(), 
required=False),
+        NestedField(field_id=11, name='fare_amount', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=12, name='extra', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=13, name='mta_tax', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=14, name='tip_amount', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=15, name='tolls_amount', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=16, name='improvement_surcharge', 
field_type=DoubleType(), required=False),
+        NestedField(field_id=17, name='total_amount', field_type=DoubleType(), 
required=False),
+        NestedField(field_id=18, name='congestion_surcharge', 
field_type=DoubleType(), required=False),
+        NestedField(field_id=19, name='airport_fee', field_type=DoubleType(), 
required=False)
+      ),
+      schema_id=0,
+      identifier_field_ids=[]
+    )],
+    current_schema_id=0,
+    partition_specs=[PartitionSpec(spec_id=0)],
+    default_spec_id=0,
+    last_partition_id=999,
+    properties={
+      'owner': 'root',
+      'write.format.default': 'parquet'
+    },
+    current_snapshot_id=8334458494559715805,
+    snapshots=[
+      Snapshot(
+        snapshot_id=7910949481055846233,
+        parent_snapshot_id=None,
+        sequence_number=None,
+        timestamp_ms=1662489306555,
+        
manifest_list='s3a://warehouse/wh/nyc.db/taxis/metadata/snap-7910949481055846233-1-3eb7a2e1-5b7a-4e76-a29a-3e29c176eea4.avro',
+        summary=Summary(
+          Operation.APPEND,
+          **{
+            'spark.app.id': 'local-1662489289173',
+            '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=[
+      SnapshotLogEntry(
+        snapshot_id='7910949481055846233',
+        timestamp_ms=1662489306555
+      )
+    ],
+    metadata_log=[
+      MetadataLogEntry(
+        
metadata_file='s3a://warehouse/wh/nyc.db/taxis/metadata/00000-b58341ba-6a63-4eea-9b2f-e85e47c7d09f.metadata.json',
+        timestamp_ms=1662489306555
+      )
+    ],
+    sort_orders=[SortOrder(order_id=0)],
+    default_sort_order_id=0,
+    refs={
+      'main': SnapshotRef(
+        snapshot_id=8334458494559715805,
+        snapshot_ref_type=SnapshotRefType.BRANCH,
+        min_snapshots_to_keep=None,
+        max_snapshot_age_ms=None,
+        max_ref_age_ms=None
+      )
+    },
+    format_version=2,
+    last_sequence_number=1
+  )
+)
+```
+
+## Create a table
+
+To create a table from a catalog:
+
+```python
+from pyiceberg.catalog import load_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.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()))
+
+catalog = load_catalog("prod")
+
+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:
+
+```python
+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
+    )
+)
+```
+
+## Query a table
+
+To query a table, a table scan is needed. A table scan accepts a filter, 
columns and optionally a snapshot ID:
+
+```python
+from pyiceberg.catalog import load_catalog
+from pyiceberg.expressions import GreaterThanOrEqual
+
+catalog = load_catalog("default")
+table = catalog.load_table("nyc.taxis")
+
+scan = table.scan(
+    row_filter=GreaterThanOrEqual("trip_distance", 10.0),
+    selected_fields=("VendorID", "tpep_pickup_datetime", 
"tpep_dropoff_datetime"),
+)
+
+[task.file.file_path for task in scan.plan_files()]
+```
+
+The low level API `plan_files` methods returns a set of tasks that provide the 
files that might contain matching rows:
+
+```json
+['s3a://warehouse/wh/nyc/taxis/data/00003-4-42464649-92dd-41ad-b83b-dea1a2fe4b58-00001.parquet']
+```
+
+In this case it is up to the engine itself to filter the file itself. Below, 
`to_arrow()` and `to_duckdb()` that already do this for you.
+
+### Apache Arrow
+
+!!! note "Requirements"
+    This requires [PyArrow to be installed](index.md)
+
+Using PyIceberg it is filter out data from a huge table and pull it into a 
PyArrow table:
+
+```python
+table.scan(
+    row_filter=GreaterThanOrEqual("trip_distance", 10.0),
+    selected_fields=("VendorID", "tpep_pickup_datetime", 
"tpep_dropoff_datetime"),
+).to_arrow()
+```
+
+This will return a PyArrow table:
+
+```
+pyarrow.Table
+VendorID: int64
+tpep_pickup_datetime: timestamp[us, tz=+00:00]
+tpep_dropoff_datetime: timestamp[us, tz=+00:00]
+----
+VendorID: 
[[2,1,2,1,1,...,2,2,2,2,2],[2,1,1,1,2,...,1,1,2,1,2],...,[2,2,2,2,2,...,2,6,6,2,2],[2,2,2,2,2,...,2,2,2,2,2]]
+tpep_pickup_datetime: [[2021-04-01 00:28:05.000000,...,2021-04-30 
23:44:25.000000]]
+tpep_dropoff_datetime: [[2021-04-01 00:47:59.000000,...,2021-05-01 
00:14:47.000000]]
+```
+
+This will only pull in the files that that might contain matching rows.
+
+### DuckDB
+
+!!! note "Requirements"
+    This requires [DuckDB to be installed](index.md).
+
+A table scan can also be converted into a in-memory DuckDB table:
+
+```python
+con = table.scan(
+    row_filter=GreaterThanOrEqual("trip_distance", 10.0),
+    selected_fields=("VendorID", "tpep_pickup_datetime", 
"tpep_dropoff_datetime"),
+).to_duckdb(table_name="distant_taxi_trips")
+```
+
+Using the cursor that we can run queries on the DuckDB table:
+
+```python
+print(
+    con.execute(
+        "SELECT tpep_dropoff_datetime - tpep_pickup_datetime AS duration FROM 
distant_taxi_trips LIMIT 4"
+    ).fetchall()
+)
+[
+    (datetime.timedelta(seconds=1194),),
+    (datetime.timedelta(seconds=1118),),
+    (datetime.timedelta(seconds=1697),),
+    (datetime.timedelta(seconds=1581),),
+]
+```
diff --git a/python/mkdocs/docs/cli.md b/python/mkdocs/docs/cli.md
new file mode 100644
index 0000000000..5b760e1fa6
--- /dev/null
+++ b/python/mkdocs/docs/cli.md
@@ -0,0 +1,215 @@
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -   http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing,
+  - software distributed under the License is distributed on an
+  - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  - KIND, either express or implied.  See the License for the
+  - specific language governing permissions and limitations
+  - under the License.
+  -->
+
+# Python CLI
+
+Pyiceberg comes with a CLI that's available after installing the `pyiceberg` 
package.
+
+You can pass the path to the Catalog using the `--uri` and `--credential` 
argument, but it is recommended to setup a `~/.pyiceberg.yaml` config as 
described in the [Catalog](configuration.md) section.
+
+```sh
+➜  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
+```
+
+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`.
+
+```sh
+➜  pyiceberg list
+default
+nyc
+```
+
+```sh
+➜  pyiceberg list nyc
+nyc.taxis
+```
+
+```sh
+➜  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:
+
+```sh
+➜  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": []
+  }
+}
+```
diff --git a/python/mkdocs/docs/configuration.md 
b/python/mkdocs/docs/configuration.md
new file mode 100644
index 0000000000..985d258173
--- /dev/null
+++ b/python/mkdocs/docs/configuration.md
@@ -0,0 +1,84 @@
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -   http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing,
+  - software distributed under the License is distributed on an
+  - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  - KIND, either express or implied.  See the License for the
+  - specific language governing permissions and limitations
+  - under the License.
+  -->
+
+# Catalogs
+
+PyIceberg currently has native support for REST, Hive and Glue.
+
+There are three ways to pass in configuration:
+
+- Using the `~/.pyiceberg.yaml` configuration file
+- Through environment variables
+- By passing in credentials through the CLI or the Python API
+
+The configuration file is recommended since that's the most transparent way. 
If you prefer environment configuration:
+
+```sh
+export PYICEBERG_CATALOG__DEFAULT__URI=thrift://localhost:9083
+```
+
+The environment variable picked up by Iceberg starts with `PYICEBERG_` and 
then follows the yaml structure below, where a double underscore `__` 
represents a nested field.
+
+For the FileIO there are several configuration options available:
+
+| Key                  | Example             | Description                     
                                                                                
                                                                                
                                                          |
+|----------------------|---------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| s3.endpoint          | https://10.0.19.25/ | Configure an alternative 
endpoint of the S3 service for the FileIO to access. This could be used to use 
S3FileIO with any s3-compatible object storage service that has a different 
endpoint, or access a private S3 endpoint in a virtual private cloud. |
+| s3.access-key-id     | admin               | Configure the static secret 
access key used to access the FileIO.                                           
                                                                                
                                                              |
+| s3.secret-access-key | password            | Configure the static session 
token used to access the FileIO.                                                
                                                                                
                                                             |
+| s3.signer            | bearer              | Configure the signature version 
of the FileIO.                                                                  
                                                                                
                                                          |
+
+## REST Catalog
+
+```yaml
+catalog:
+  default:
+    uri: http://rest-catalog/ws/
+    credential: t-1234:secret
+
+  default-mtls-secured-catalog:
+    uri: https://rest-catalog/ws/
+    ssl:
+      client:
+        cert: /absolute/path/to/client.crt
+        key: /absolute/path/to/client.key
+      cabundle: /absolute/path/to/cabundle.pem
+```
+
+## Hive Catalog
+
+```yaml
+catalog:
+  default:
+    uri: thrift://localhost:9083
+    s3.endpoint: http://localhost:9000
+    s3.access-key-id: admin
+    s3.secret-access-key: password
+```
+
+## Glue Catalog
+
+If you want to use AWS Glue as the catalog, you can use the last two ways to 
configure the pyiceberg and refer
+[How to configure AWS 
credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html)
 to set your AWS account credentials locally.
+
+```yaml
+catalog:
+  default:
+    type: glue
+```
diff --git a/python/mkdocs/docs/contributing.md 
b/python/mkdocs/docs/contributing.md
new file mode 100644
index 0000000000..a87939935f
--- /dev/null
+++ b/python/mkdocs/docs/contributing.md
@@ -0,0 +1,135 @@
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -   http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing,
+  - software distributed under the License is distributed on an
+  - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  - KIND, either express or implied.  See the License for the
+  - specific language governing permissions and limitations
+  - under the License.
+  -->
+
+# Contributing to the Iceberg Python library
+
+For the development, poetry is used for packing and dependency management. You 
can install this using:
+
+```bash
+pip install poetry
+```
+
+If you have an older version of pip and virtualenv you need to update these:
+
+```bash
+pip install --upgrade virtualenv pip
+```
+
+To get started, you can run `make install`, which will install poetry and it 
will install all the dependencies of the Iceberg library. This will also 
install the development dependencies. If you don't want to do this, you need to 
install using `poetry install --no-dev`.
+
+If you want to install the library on the host, you can simply run `pip3 
install -e .`. If you wish to use a virtual environment, you can run `poetry 
shell`. Poetry will open up a virtual environment with all the dependencies set.
+
+To set up IDEA with Poetry ([also on 
Loom](https://www.loom.com/share/6d36464d45f244729d91003e7f671fd2)):
+
+- Open up the Python project in IntelliJ
+- Make sure that you're on latest master (that includes Poetry)
+- Go to File -> Project Structure (⌘;)
+- Go to Platform Settings -> SDKs
+- Click the + sign -> Add Python SDK
+- Select Poetry Environment from the left hand side bar and hit OK
+- It can take some time to download all the dependencies based on your internet
+- Go to Project Settings -> Project
+- Select the Poetry SDK from the SDK dropdown, and click OK
+
+For IDEA ≤2021 you need to install the [Poetry integration as a 
plugin](https://plugins.jetbrains.com/plugin/14307-poetry/).
+
+Now you're set using Poetry, and all the tests will run in Poetry, and you'll 
have syntax highlighting in the pyproject.toml to indicate stale dependencies.
+
+## Linting
+
+`pre-commit` is used for autoformatting and linting:
+
+```bash
+make lint
+```
+
+Pre-commit will automatically fix the violations such as import orders, 
formatting etc. Pylint errors you need to fix yourself.
+
+In contrast to the name suggest, it doesn't run the checks on the commit. If 
this is something that you like, you can set this up by running `pre-commit 
install`.
+
+You can bump the integrations to the latest version using `pre-commit 
autoupdate`. This will check if there is a newer version of 
`{black,mypy,isort,...}` and update the yaml.
+
+## Testing
+
+For Python, `pytest` is used a testing framework in combination with 
`coverage` to enforce 90%+ code coverage.
+
+```bash
+make test
+```
+
+By default, s3 tests are ignored because that require minio to be running. To 
run the s3 suite:
+
+```bash
+make test-s3
+```
+
+To pass additional arguments to pytest, you can use `PYTEST_ARGS`.
+
+*Run pytest in verbose mode*
+
+```sh
+make test PYTEST_ARGS="-v"
+```
+
+*Run pytest with pdb enabled*
+
+```sh
+make test PYTEST_ARGS="--pdb"
+```
+
+To see all available pytest arguments, run `make test PYTEST_ARGS="--help"`.
+
+## Code standards
+
+Below are the formalized conventions that we adhere to in the PyIceberg 
project. The goal of this is to have a common agreement on how to evolve the 
codebase, but also using it as guidelines for newcomers to the project.
+
+## API Compatibility
+
+It is important to keep the Python public API compatible across versions. The 
Python official [PEP-8](https://peps.python.org/pep-0008/) defines Public 
methods as: _Public attributes should have no leading underscores_. This means 
not removing any methods without any notice, or removing or renaming any 
existing parameters. Adding new optional parameters is okay.
+
+If you want to remove a method, please add a deprecation notice by annotating 
the function using `@deprecated`:
+
+```python
+from pyiceberg.utils.deprecated import deprecated
+
+
+@deprecated(
+    deprecated_in="0.1.0",
+    removed_in="0.2.0",
+    help_message="Please use load_something_else() instead",
+)
+def load_something():
+    pass
+```
+
+Which will warn:
+
+```
+Call to load_something, deprecated in 0.1.0, will be removed in 0.2.0. Please 
use load_something_else() instead.
+```
+
+## Type annotations
+
+For the type annotation the types from the `Typing` package are used.
+
+PyIceberg offers support from Python 3.8 onwards, we can't use the [type hints 
from the standard collections](https://peps.python.org/pep-0585/).
+
+## Third party libraries
+
+PyIceberg naturally integrates into the rich Python ecosystem, however it is 
important to be hesistant to add third party packages. Adding a lot of packages 
makes the library heavyweight, and causes incompatibilities with other projects 
if they use a different version of the library. Also, big libraries such as 
`s3fs`, `pyarrow`, `thrift` should be optional to avoid downloading everything, 
while not being sure if is actually being used.
diff --git a/python/mkdocs/docs/feature-support.md 
b/python/mkdocs/docs/feature-support.md
new file mode 100644
index 0000000000..366dfbe0c5
--- /dev/null
+++ b/python/mkdocs/docs/feature-support.md
@@ -0,0 +1,85 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+# 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  |   X    |
+| Plan Scan for Snapshot   |    X  |   X    |
+| Update Current Snapshot  |    X  |        |
+| Create Table             |    X  |   X    |
+| Rename Table             |    X  |   X    |
+| Drop Table               |    X  |   X    |
+| Alter Table              |    X  |        |
+| Set Table Properties     |    X  |        |
+| Create Namespace         |    X  |   X    |
+| Drop Namespace           |    X  |   X    |
+| Set Namespace Properties |    X  |   X    |
+
+## Types
+
+The types are kept 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)`
+
+## Expressions
+
+The expressions are kept in `pyiceberg.expressions`.
+
+- `IsNull`
+- `NotNull`
+- `IsNaN`
+- `NotNaN`
+- `In`
+- `NotIn`
+- `EqualTo`
+- `NotEqualTo`
+- `GreaterThanOrEqual`
+- `GreaterThan`
+- `LessThanOrEqual`
+- `LessThan`
+- `And`
+- `Or`
+- `Not`
diff --git a/python/dev/RELEASE.md b/python/mkdocs/docs/how-to-release.md
similarity index 79%
rename from python/dev/RELEASE.md
rename to python/mkdocs/docs/how-to-release.md
index a99683031c..5f3b6795a9 100644
--- a/python/dev/RELEASE.md
+++ b/python/mkdocs/docs/how-to-release.md
@@ -21,7 +21,7 @@
 
 The guide to release PyIceberg.
 
-First we're going to release a release candidate (RC) and publish it to the 
public for testing and validation. Once the vote has passed on the RC, we can 
release the new version.
+The first step is to publish a release candidate (RC) and publish it to the 
public for testing and validation. Once the vote has passed on the RC, the RC 
turns into the new release.
 
 ## Running a release candidate
 
@@ -44,7 +44,7 @@ export LAST_COMMIT_ID=$(git rev-list ${GIT_TAG} 2> /dev/null 
| head -n 1)
 
 The `-s` option will sign the commit. If you don't have a key yet, you can 
find the instructions 
[here](http://www.apache.org/dev/openpgp.html#key-gen-generate-key). To install 
gpg on a M1 based Mac, a couple of additional steps are required: 
https://gist.github.com/phortuin/cf24b1cca3258720c71ad42977e1ba57
 
-Next we'll create a source distribution (`sdist`) which will generate a 
`.tar.gz` with all the source files. So we can upload the files to the Apache 
SVN.
+Next step is to create a source distribution (`sdist`) which will generate a 
`.tar.gz` with all the source files. These files need to be uploaded to the 
Apache SVN.
 
 ```
 poetry build
@@ -62,7 +62,7 @@ Building pyiceberg (0.1.0)
 
 The `sdist` contains the source which can be used for checking licenses, and 
the wheel is a compiled version for quick installation.
 
-Before committing the files to the Apache SVN artifact distribution SVN, we 
need to generate hashes, and we need to sign them using gpg:
+Before committing the files to the Apache SVN artifact distribution SVN hashes 
need to be generated, and those need to be signed with gpg to make sure that 
they are authentic:
 
 ```bash
 for name in "pyiceberg-${VERSION_WITHOUT_RC}-py3-none-any.whl" 
"pyiceberg-${VERSION_WITHOUT_RC}.tar.gz"
@@ -72,7 +72,7 @@ do
 done
 ```
 
-Next, we'll clone the Apache SVN, copy and commit the files:
+Next step is to clone the Apache SVN, copy and commit the files:
 
 ```bash
 export SVN_TMP_DIR=/tmp/iceberg-${VERSION_BRANCH}/
@@ -85,7 +85,7 @@ svn add $SVN_TMP_DIR_VERSIONED
 svn ci -m "PyIceberg ${VERSION}" ${SVN_TMP_DIR_VERSIONED}
 ```
 
-Next, we can upload them to pypi. Please keep in mind that this **won't** bump 
the version for everyone that hasn't pinned their version, we set it to a RC 
[pre-release and those are 
ignored](https://packaging.python.org/en/latest/guides/distributing-packages-using-setuptools/#pre-release-versioning).
+Next step is to upload them to pypi. Please keep in mind that this **won't** 
bump the version for everyone that hasn't pinned their version, since it is set 
to a RC [pre-release and those are 
ignored](https://packaging.python.org/en/latest/guides/distributing-packages-using-setuptools/#pre-release-versioning).
 
 ```
 poetry version ${VERSION}
@@ -94,7 +94,7 @@ poetry build
 twine upload -s dist/*
 ```
 
-Finally, we can generate the email what we'll send to the mail list:
+Finally step is to generate the email what send to the dev mail list:
 
 ```bash
 cat << EOF > release-announcement-email.txt
@@ -134,3 +134,13 @@ EOF
 
 cat release-announcement-email.txt
 ```
+
+## Vote has passed
+
+Once the vote has been passed, the latest version can be pushed to PyPi. Check 
out the commit associated with the passing vote, and run:
+
+```bash
+rm -rf dist/
+poetry build
+twine upload -s dist/*
+```
diff --git a/python/mkdocs/docs/index.md b/python/mkdocs/docs/index.md
index 53795a8e07..e36588e434 100644
--- a/python/mkdocs/docs/index.md
+++ b/python/mkdocs/docs/index.md
@@ -17,9 +17,9 @@
 
 # PyIceberg
 
-Much of the python api conforms to the Java API. You can get more info about 
the java api 
[here](https://iceberg.apache.org/docs/latest/java-api-quickstart/).
+PyIceberg is a Python implementation for accessing Iceberg tables, without the 
need of a JVM.
 
-## Installing
+## Install
 
 You can install the latest release version from pypi:
 
@@ -27,543 +27,31 @@ You can install the latest release version from pypi:
 pip3 install "pyiceberg[s3fs,hive]"
 ```
 
-Or install the latest development version locally:
+Install it directly for Github (not recommended), but sometimes handy:
 
-```sh
-git clone https://github.com/apache/iceberg.git
-cd iceberg/python
-pip3 install -e ".[s3fs,hive]"
 ```
-
-You can mix and match optional dependencies:
-
-| Key       | Description:                                                     
    |
-|-----------|----------------------------------------------------------------------|
-| hive      | Support for the Hive metastore                                   
    |
-| glue      | Support for AWS Glue                                             
    |
-| 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 comes with a CLI that's available after installing the `pyiceberg` 
package.
-
-```sh
-➜  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
+pip install 
"git+https://github.com/apache/iceberg.git#subdirectory=python&egg=pyiceberg[s3fs]";
 ```
 
-# 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`:
-
-```yaml
-catalog:
-  default:
-    uri: thrift://localhost:9083
-    s3.endpoint: http://localhost:9000
-    s3.access-key-id: admin
-    s3.secret-access-key: password
-
-  rest:
-    uri: http://rest-catalog/ws/
-    credential: t-1234:secret
-
-  mtls-secured-catalog:
-    uri: https://rest-catalog/ws/
-    ssl:
-      client:
-        cert: /absolute/path/to/client.crt
-        key: /absolute/path/to/client.key
-      cabundle: /absolute/path/to/cabundle.pem
-
-  glue:
-    type: glue
-```
-
-Lastly, you can also set it using environment variables:
+Or clone the repository for local development:
 
 ```sh
-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
-
-export PYICEBERG_CATALOG__GLUE__TYPE=glue
-```
-
-Where the structure is equivalent to the YAML. The levels are separated using 
a double underscore (`__`).
-
-If you want to use AWS Glue as the catalog, you can use the last two ways to 
configure the pyiceberg and refer
-[How to configure AWS 
credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html)
 to set your AWS account credentials locally.
-
-## FileIO configuration
-
-For the FileIO there are several configuration options available:
-
-| Key                  | Example             | Description                     
                                                                                
                                                                                
                                                          |
-|----------------------|---------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| s3.endpoint          | https://10.0.19.25/ | Configure an alternative 
endpoint of the S3 service for the FileIO to access. This could be used to use 
S3FileIO with any s3-compatible object storage service that has a different 
endpoint, or access a private S3 endpoint in a virtual private cloud. |
-| s3.access-key-id     | admin               | Configure the static secret 
access key used to access the FileIO.                                           
                                                                                
                                                              |
-| s3.secret-access-key | password            | Configure the static session 
token used to access the FileIO.                                                
                                                                                
                                                             |
-| s3.signer            | bearer              | Configure the signature version 
of the FileIO.                                                                  
                                                                                
                                                          |
-
-# 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`.
-
-```sh
-➜  pyiceberg list
-default
-nyc
-```
-
-```sh
-➜  pyiceberg list nyc
-nyc.taxis
-```
-
-```sh
-➜  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:
-
-```sh
-➜  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:
-
-```python
-from pyiceberg.catalog import load_catalog
-
-catalog = load_catalog("prod")
-
-catalog.list_namespaces()
-```
-
-Returns:
-
-```
-[('default',), ('nyc',)]
-```
-
-Listing the tables in the `nyc` namespace:
-
-```python
-catalog.list_tables("nyc")
-```
-
-Returns:
-
-```
-[('nyc', 'taxis')]
-```
-
-Loading the `taxis` table:
-
-```python
-catalog.load_table(("nyc", "taxis"))
-```
-
-```
-Table(
-  identifier=('nyc', 'taxis'),
-  
metadata_location='s3a://warehouse/wh/nyc.db/taxis/metadata/00002-6ea51ce3-62aa-4197-9cf8-43d07c3440ca.metadata.json',
-  metadata=TableMetadataV2(
-    location='s3a://warehouse/wh/nyc.db/taxis',
-    table_uuid=UUID('ebd5d172-2162-453d-b586-1cdce52c1116'),
-    last_updated_ms=1662633437826,
-    last_column_id=19,
-    schemas=[Schema(
-        NestedField(field_id=1, name='VendorID', field_type=LongType(), 
required=False),
-        NestedField(field_id=2, name='tpep_pickup_datetime', 
field_type=TimestamptzType(), required=False),
-        NestedField(field_id=3, name='tpep_dropoff_datetime', 
field_type=TimestamptzType(), required=False),
-        NestedField(field_id=4, name='passenger_count', 
field_type=DoubleType(), required=False),
-        NestedField(field_id=5, name='trip_distance', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=6, name='RatecodeID', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=7, name='store_and_fwd_flag', 
field_type=StringType(), required=False),
-        NestedField(field_id=8, name='PULocationID', field_type=LongType(), 
required=False),
-        NestedField(field_id=9, name='DOLocationID', field_type=LongType(), 
required=False),
-        NestedField(field_id=10, name='payment_type', field_type=LongType(), 
required=False),
-        NestedField(field_id=11, name='fare_amount', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=12, name='extra', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=13, name='mta_tax', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=14, name='tip_amount', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=15, name='tolls_amount', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=16, name='improvement_surcharge', 
field_type=DoubleType(), required=False),
-        NestedField(field_id=17, name='total_amount', field_type=DoubleType(), 
required=False),
-        NestedField(field_id=18, name='congestion_surcharge', 
field_type=DoubleType(), required=False),
-        NestedField(field_id=19, name='airport_fee', field_type=DoubleType(), 
required=False)
-      ),
-      schema_id=0,
-      identifier_field_ids=[]
-    )],
-    current_schema_id=0,
-    partition_specs=[PartitionSpec(spec_id=0)],
-    default_spec_id=0,
-    last_partition_id=999,
-    properties={
-      'owner': 'root',
-      'write.format.default': 'parquet'
-    },
-    current_snapshot_id=8334458494559715805,
-    snapshots=[
-      Snapshot(
-        snapshot_id=7910949481055846233,
-        parent_snapshot_id=None,
-        sequence_number=None,
-        timestamp_ms=1662489306555,
-        
manifest_list='s3a://warehouse/wh/nyc.db/taxis/metadata/snap-7910949481055846233-1-3eb7a2e1-5b7a-4e76-a29a-3e29c176eea4.avro',
-        summary=Summary(
-          Operation.APPEND,
-          **{
-            'spark.app.id': 'local-1662489289173',
-            '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=[
-      SnapshotLogEntry(
-        snapshot_id='7910949481055846233',
-        timestamp_ms=1662489306555
-      )
-    ],
-    metadata_log=[
-      MetadataLogEntry(
-        
metadata_file='s3a://warehouse/wh/nyc.db/taxis/metadata/00000-b58341ba-6a63-4eea-9b2f-e85e47c7d09f.metadata.json',
-        timestamp_ms=1662489306555
-      )
-    ],
-    sort_orders=[SortOrder(order_id=0)],
-    default_sort_order_id=0,
-    refs={
-      'main': SnapshotRef(
-        snapshot_id=8334458494559715805,
-        snapshot_ref_type=SnapshotRefType.BRANCH,
-        min_snapshots_to_keep=None,
-        max_snapshot_age_ms=None,
-        max_ref_age_ms=None
-      )
-    },
-    format_version=2,
-    last_sequence_number=1
-  )
-)
-```
-
-And to create a table from a catalog:
-
-```python
-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.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
-    )
-)
+git clone https://github.com/apache/iceberg.git
+cd iceberg/python
+pip3 install -e ".[s3fs,hive]"
 ```
 
-# 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  |   X    |
-| Plan Scan for Snapshot   |    X  |   X    |
-| Update Current Snapshot  |    X  |        |
-| Create Table             |    X  |   X    |
-| Rename Table             |    X  |   X    |
-| Drop Table               |    X  |   X    |
-| Alter Table              |    X  |        |
-| Set Table Properties     |    X  |        |
-| Create Namespace         |    X  |   X    |
-| Drop Namespace           |    X  |   X    |
-| Set Namespace Properties |    X  |   X    |
-
-## Types
-
-The types are kept in `pyiceberg.types`.
-
-Primitive types:
+You can mix and match optional dependencies depending on your needs:
 
-- `BooleanType`
-- `StringType`
-- `IntegerType`
-- `LongType`
-- `FloatType`
-- `DoubleType`
-- `DateType`
-- `TimeType`
-- `TimestampType`
-- `TimestamptzType`
-- `BinaryType`
-- `UUIDType`
+| Key     | Description:                                                       
  |
+|---------|----------------------------------------------------------------------|
+| hive    | Support for the Hive metastore                                     
  |
+| glue    | Support for AWS Glue                                               
  |
+| pyarrow | PyArrow as a FileIO implementation to interact with the object 
store |
+| duckdb  | Installs both PyArrow and DuckDB                                   
  |
+| s3fs    | S3FS as a FileIO implementation to interact with the object store  
  |
+| snappy  | Support for snappy Avro compression                                
  |
 
-Complex types:
+You either need to install `s3fs` or `pyarrow` for fetching files.
 
-- `StructType`
-- `ListType`
-- `MapType`
-- `FixedType(16)`
-- `DecimalType(8, 3)`
+There is both a [CLI](cli.md) and [Python API](api.md) available.
diff --git a/python/mkdocs/docs/verify-release.md 
b/python/mkdocs/docs/verify-release.md
new file mode 100644
index 0000000000..e453eea12e
--- /dev/null
+++ b/python/mkdocs/docs/verify-release.md
@@ -0,0 +1,100 @@
+<!--
+  - Licensed to the Apache Software Foundation (ASF) under one
+  - or more contributor license agreements.  See the NOTICE file
+  - distributed with this work for additional information
+  - regarding copyright ownership.  The ASF licenses this file
+  - to you under the Apache License, Version 2.0 (the
+  - "License"); you may not use this file except in compliance
+  - with the License.  You may obtain a copy of the License at
+  -
+  -   http://www.apache.org/licenses/LICENSE-2.0
+  -
+  - Unless required by applicable law or agreed to in writing,
+  - software distributed under the License is distributed on an
+  - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  - KIND, either express or implied.  See the License for the
+  - specific language governing permissions and limitations
+  - under the License.
+  -->
+
+# Verifying a release
+
+Each Apache PyIceberg release is validated by the community by holding a vote. 
A community release manager will prepare a release candidate and call a vote on 
the Iceberg dev list. To validate the release candidate, community members will 
test it out in their downstream projects and environments.
+
+In addition to testing in downstream projects, community members also check 
the release’s signatures, checksums, and license documentation.
+
+## Validating a release candidate
+
+Release announcements include links to the following:
+
+- A source tarball
+- A signature (.asc)
+- A checksum (.sha512)
+- KEYS file
+- GitHub change comparison
+
+After downloading the source tarball, signature, checksum, and KEYS file, here 
are instructions on how to verify signatures, checksums, and documentation.
+
+## Verifying signatures
+
+First, import the keys.
+
+```sh
+curl https://dist.apache.org/repos/dist/dev/iceberg/KEYS -o KEYS
+gpg --import KEYS
+```
+
+Next, verify the `.asc` file.
+
+```sh
+gpg --verify pyiceberg-0.2.0rc0-py3-none-any.whl.asc 
pyiceberg-0.2.0rc0-py3-none-any.whl
+```
+
+## Verifying checksums
+
+```sh
+shasum -a 512 pyiceberg-0.2.0rc0.tar.gz
+```
+
+## Verifying License Documentation
+
+```sh
+tar xzf pyiceberg-0.2.0rc0.tar.gztar
+cd pyiceberg-0.2.0
+```
+
+Run RAT checks to validate license header:
+
+```
+./dev/check-license
+```
+
+## Testing
+
+First step is to install the package:
+
+```sh
+make install
+```
+
+And then run the tests:
+
+```sh
+make test
+```
+
+To run the full integration tests:
+
+```sh
+make test-s3
+```
+
+This will include a Minio S3 container being spun up.
+
+# Cast the vote
+
+Votes are cast by replying to the release candidate announcement email on the 
dev mailing list with either `+1`, `0`, or `-1`.
+
+> \[ \] +1 Release this as Apache Iceberg 1.1.0 \[ \] +0 \[ \] -1 Do not 
release this because…
+
+In addition to your vote, it’s customary to specify if your vote is binding or 
non-binding. Only members of the Project Management Committee have formally 
binding votes. If you’re unsure, you can specify that your vote is non-binding. 
To read more about voting in the Apache framework, checkout the 
[Voting](https://www.apache.org/foundation/voting.html) information page on the 
Apache foundation’s website.
diff --git a/python/mkdocs/mkdocs.yml b/python/mkdocs/mkdocs.yml
index c84a2de465..522a45342e 100644
--- a/python/mkdocs/mkdocs.yml
+++ b/python/mkdocs/mkdocs.yml
@@ -16,3 +16,19 @@
 # under the License.
 ---
 site_name: PyIceberg
+site_url: https://py.iceberg.apache.org/
+nav:
+    - Home: index.md
+    - Configuration: configuration.md
+    - CLI: cli.md
+    - API: api.md
+    - Contributing:
+        - Contributing: contributing.md
+    - Feature support: feature-support.md
+    - Releases:
+        - Verify a release: verify-release.md
+        - How to release: how-to-release.md
+theme:
+    name: readthedocs
+markdown_extensions:
+    - admonition

Reply via email to