You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/24 19:21:08 UTC

[GitHub] [iceberg] Fokko opened a new pull request, #4858: Python: Add generated classes from OpenAPI spec

Fokko opened a new pull request, #4858:
URL: https://github.com/apache/iceberg/pull/4858

   This makes is typesafe to generate the classes for the Rest Catalog spec


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1143941149

   The abovementioned issue has been fixed in https://github.com/apache/iceberg/pull/4899
   
   Based on the discussion at the [Python sync](https://docs.google.com/document/d/1oMKodaZJrOJjPfc8PDVAoTdl02eGQKHlhwuggiw7s9U/edit), let's close this for now. We can't integrate the code into our existing codebase, and we don't want to maintain the two separately :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#discussion_r881892637


##########
.github/workflows/openapi-ci.yml:
##########
@@ -0,0 +1,51 @@
+#
+# 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.
+#
+
+name: "OpenAPI CI"
+on:
+  push:
+    branches:
+      - 'master'
+      - '0.**'
+    tags:
+      - 'apache-iceberg-**'
+  pull_request:
+    paths:
+      - '.github/workflows/openapi-ci.yml'
+      - 'openapi/**'
+
+concurrency:
+  group: ${{ github.workflow }}-${{ github.ref }}
+  cancel-in-progress: ${{ github.event_name == 'pull_request' }}
+
+jobs:
+  openapi:
+    runs-on: ubuntu-20.04
+    steps:
+      - uses: actions/checkout@v2
+      - uses: actions/setup-python@v2
+        with:
+          python-version: 3.9
+      - working-directory: ./python

Review Comment:
   Thanks, that got dropped somewhere 👍🏻 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#discussion_r883860738


##########
python/src/iceberg/openapi/rest_catalog.py:
##########
@@ -0,0 +1,503 @@
+# 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.
+# generated by datamodel-codegen:
+#   filename:  rest-catalog-open-api.yaml
+
+from __future__ import annotations
+
+from enum import Enum
+from typing import Any, Dict, List, Literal, Optional, Union
+
+from pydantic import BaseModel, Extra, Field
+
+
+class ErrorModel(BaseModel):
+    """
+    JSON error payload returned in a response with further details on the error
+    """
+
+    message: str = Field(..., description='Human-readable error message')
+    type: str = Field(
+        ...,
+        description='Internal type definition of the error',
+        example='NoSuchNamespaceException',
+    )
+    code: int = Field(
+        ..., description='HTTP response code', example=404, ge=400, le=600
+    )
+    stack: Optional[List[str]] = None
+
+
+class CatalogConfig(BaseModel):
+    """
+    Server-provided configuration for the catalog.
+    """
+
+    overrides: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used to override client configuration; applied after defaults and client configuration.',
+    )
+    defaults: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used as default configuration; applied before client configuration.',
+    )
+
+
+class Updates(BaseModel):
+    pass
+
+
+class UpdateNamespacePropertiesRequest(BaseModel):
+    removals: Optional[List[str]] = Field(
+        None, example=['department', 'access_group'], unique_items=True
+    )
+    updates: Optional[Union[List[str], Updates]] = Field(
+        None, example={'owner': 'Hank Bendickson'}, unique_items=True
+    )
+
+
+class Namespace(BaseModel):
+    """
+    Reference to one or more levels of a namespace
+    """
+
+    __root__: List[str] = Field(
+        ...,
+        description='Reference to one or more levels of a namespace',
+        example=['accounting', 'tax'],
+    )
+
+
+class TableIdentifier(BaseModel):
+    namespace: Namespace
+    name: str
+
+
+class PrimitiveType(BaseModel):
+    __root__: str = Field(..., example=['long', 'string', 'fixed[16]', 'decimal(10,2)'])
+
+
+class Transform(BaseModel):
+    __root__: str = Field(
+        ...,
+        example=[
+            'identity',
+            'year',
+            'month',
+            'day',
+            'hour',
+            'bucket[256]',
+            'truncate[16]',
+        ],
+    )
+
+
+class PartitionField(BaseModel):
+    field_id: Optional[int] = Field(None, alias='field-id')

Review Comment:
   We would need to use the proper name (kebab case `field-id`) any time it's serialized, be that stored in a file or sent over the network for things like the REST catalog etc.
   
   I don't personally think it's a good idea to start allowing for multiple forms accepted when stored in files or over the network, as we'll always have to support those then and it adds unnecessary complexity. We have some places where we have additional logic for things like 3-level lists in parquet (as their form changed during some versions before). So once those files are written, we _always_ have to support that alternative form or we have to make the choice to break people's existing tables (that they might have not touched for a while).
   
   So generally that's something we'd avoid when serializing within files or within REST requests.
   
   Otherwise, within the Python project and code itself, it is fine to use underscore and other things. E.g. Java doesn't allow for `field-id` as an identifier so we would generally use `fieldId`.
   
   We'd just want to be sure that we test that the JSON is always used correctly (either via some helper that ensures `by_alias` is used when needed or just extensive testing).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#discussion_r883860738


##########
python/src/iceberg/openapi/rest_catalog.py:
##########
@@ -0,0 +1,503 @@
+# 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.
+# generated by datamodel-codegen:
+#   filename:  rest-catalog-open-api.yaml
+
+from __future__ import annotations
+
+from enum import Enum
+from typing import Any, Dict, List, Literal, Optional, Union
+
+from pydantic import BaseModel, Extra, Field
+
+
+class ErrorModel(BaseModel):
+    """
+    JSON error payload returned in a response with further details on the error
+    """
+
+    message: str = Field(..., description='Human-readable error message')
+    type: str = Field(
+        ...,
+        description='Internal type definition of the error',
+        example='NoSuchNamespaceException',
+    )
+    code: int = Field(
+        ..., description='HTTP response code', example=404, ge=400, le=600
+    )
+    stack: Optional[List[str]] = None
+
+
+class CatalogConfig(BaseModel):
+    """
+    Server-provided configuration for the catalog.
+    """
+
+    overrides: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used to override client configuration; applied after defaults and client configuration.',
+    )
+    defaults: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used as default configuration; applied before client configuration.',
+    )
+
+
+class Updates(BaseModel):
+    pass
+
+
+class UpdateNamespacePropertiesRequest(BaseModel):
+    removals: Optional[List[str]] = Field(
+        None, example=['department', 'access_group'], unique_items=True
+    )
+    updates: Optional[Union[List[str], Updates]] = Field(
+        None, example={'owner': 'Hank Bendickson'}, unique_items=True
+    )
+
+
+class Namespace(BaseModel):
+    """
+    Reference to one or more levels of a namespace
+    """
+
+    __root__: List[str] = Field(
+        ...,
+        description='Reference to one or more levels of a namespace',
+        example=['accounting', 'tax'],
+    )
+
+
+class TableIdentifier(BaseModel):
+    namespace: Namespace
+    name: str
+
+
+class PrimitiveType(BaseModel):
+    __root__: str = Field(..., example=['long', 'string', 'fixed[16]', 'decimal(10,2)'])
+
+
+class Transform(BaseModel):
+    __root__: str = Field(
+        ...,
+        example=[
+            'identity',
+            'year',
+            'month',
+            'day',
+            'hour',
+            'bucket[256]',
+            'truncate[16]',
+        ],
+    )
+
+
+class PartitionField(BaseModel):
+    field_id: Optional[int] = Field(None, alias='field-id')

Review Comment:
   We would need to use the proper name (kebab case `field-id`) any time it's serialized, be that stored in a file or sent over the network for things like the REST catalog etc.
   
   I don't personally think it's a good idea to start allowing for multiple forms accepted when stored in files or over the network, as we'll always have to support those then and it adds unnecessary complexity. We have some places where we have additional logic for things like 3-level lists in parquet (as their form changed during some versions before). So once those files are written, we _always_ have to support that alternative form or we have to make the choice to break people's existing tables (that they might have not touched for a while).
   
   So generally that's something we'd avoid when serializing within files or within REST requests.
   
   Otherwise, within the Python project and code itself, it is fine to use underscore and other things. E.g. Java doesn't allow for `field-id` as an identifier so we would generally use `fieldId`.
   
   We'd just want to be sure that we test that the JSON is always generated and used correctly (either via some helper that ensures `by_alias` is used when needed or just extensive testing).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#discussion_r883862791


##########
python/setup.cfg:
##########
@@ -44,11 +44,13 @@ packages = find:
 python_requires = >=3.8
 install_requires =
     mmh3
+    pydantic==1.9.1

Review Comment:
   Is `pydantic` required if we were to move forward with this or is this just added in this PR for other reasons?
   
   I know that we're trying to keep the required dependencies as minimal as possible for the Python library.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1140323416

   Subclassing them in another module would be the way to go. In that class, we then add all the convenience methods and additional validation, while inheriting the (de)serialization of the actual data from the Open API spec.
   
   This would rewrite from PR https://github.com/apache/iceberg/pull/4717/
   
   https://github.com/apache/iceberg/blob/f5de18f554711b4ae00d6ebefbaa126a7f83f792/python/src/iceberg/table/partitioning.py#L48-L111
   
   To the following (tests are passing :):
   
   ```python
   from iceberg.openapi import rest_catalog
   
   
   class PartitionSpec(rest_catalog.PartitionSpec):
       """
       PartitionSpec capture the transformation from table data to partition values
       Attributes:
           table_schema(IcebergSchema): the schema of data table
       """
       # Fokko: I've aliased the schema to IcebergSchema, because we also have a schema in the open_api spec
       # This would go away later on
       table_schema: IcebergSchema = Field()
       _source_id_to_fields_map: Dict[int, List[PartitionField]] = Field(init=False)
   
       @root_validator
       def check_fields_in_schema(cls, values: Dict[str, Any]):
           schema: IcebergSchema = values['table_schema']
           source_id_to_fields_map = dict()
           for partition_field in values['fields']:
               source_column = schema.find_column_name(partition_field.source_id)
               if not source_column:
                   raise ValueError(f"Cannot find source column: {partition_field.source_id}")
               existing = source_id_to_fields_map.get(partition_field.source_id, [])
               existing.append(partition_field)
               source_id_to_fields_map[partition_field.source_id] = existing
           values["_source_id_to_fields_map"] = source_id_to_fields_map
           return values
   
       def __eq__(self, other):
           """
           Equality check on spec_id and partition fields only
           """
           return self.spec_id == other.spec_id and self.fields == other.fields
   
       def __str__(self):
           """
           PartitionSpec str method highlight the partition field only
           """
           result_str = "["
           for partition_field in self.fields:
               result_str += f"\n  {str(partition_field)}"
           if len(self.fields) > 0:
               result_str += "\n"
           result_str += "]"
           return result_str
   
       def is_unpartitioned(self) -> bool:
           return len(self.fields) < 1
   
       def fields_by_source_id(self, field_id: int) -> List[PartitionField]:
           return self._source_id_to_fields_map[field_id]
   
       def compatible_with(self, other: "PartitionSpec") -> bool:
           """
           Returns true if this partition spec is equivalent to the other, with partition field_id ignored.
           That is, if both specs have the same number of fields, field order, field name, source column ids, and transforms.
           """
           return all(
               this_field.source_id == that_field.source_id
               and this_field.transform == that_field.transform
               and this_field.name == that_field.name
               for this_field, that_field in zip(self.fields, other.fields)
           )
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1140068161

   Okay, so assuming that we go this direction, how do we integrate these generated classes with the other code that needs to be there? `PartitionSpec` is a great example, where we can use this to get a `PartitionSpec` object that we can serialize and deserialize properly, along with basic validation. But how do we expose methods like `PartitionSpec.partitionType()` as these change? Do we make subclasses?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] samredai commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
samredai commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1137526561

   We need a `iceberg/openapi/__init__.py` file right?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] samredai commented on a diff in pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
samredai commented on code in PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#discussion_r881813525


##########
.github/workflows/openapi-ci.yml:
##########
@@ -0,0 +1,51 @@
+#
+# 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.
+#
+
+name: "OpenAPI CI"
+on:
+  push:
+    branches:
+      - 'master'
+      - '0.**'
+    tags:
+      - 'apache-iceberg-**'
+  pull_request:
+    paths:
+      - '.github/workflows/openapi-ci.yml'
+      - 'openapi/**'
+
+concurrency:
+  group: ${{ github.workflow }}-${{ github.ref }}
+  cancel-in-progress: ${{ github.event_name == 'pull_request' }}
+
+jobs:
+  openapi:
+    runs-on: ubuntu-20.04
+    steps:
+      - uses: actions/checkout@v2
+      - uses: actions/setup-python@v2
+        with:
+          python-version: 3.9
+      - working-directory: ./python
+        run: |
+          make generate-openapi
+          if ! git diff --exit-code src/iceberg/openapi/rest_catalog.py; then

Review Comment:
   Awesome!



##########
.github/workflows/openapi-ci.yml:
##########
@@ -0,0 +1,51 @@
+#
+# 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.
+#
+
+name: "OpenAPI CI"
+on:
+  push:
+    branches:
+      - 'master'
+      - '0.**'
+    tags:
+      - 'apache-iceberg-**'
+  pull_request:
+    paths:
+      - '.github/workflows/openapi-ci.yml'
+      - 'openapi/**'
+
+concurrency:
+  group: ${{ github.workflow }}-${{ github.ref }}
+  cancel-in-progress: ${{ github.event_name == 'pull_request' }}
+
+jobs:
+  openapi:
+    runs-on: ubuntu-20.04
+    steps:
+      - uses: actions/checkout@v2
+      - uses: actions/setup-python@v2
+        with:
+          python-version: 3.9
+      - working-directory: ./python

Review Comment:
   Is a name here helpful? Maybe something like `name: "Validate that python OpenAPI code is in sync"`



##########
python/src/iceberg/openapi/rest_catalog.py:
##########
@@ -0,0 +1,503 @@
+# 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.
+# generated by datamodel-codegen:
+#   filename:  rest-catalog-open-api.yaml
+
+from __future__ import annotations
+
+from enum import Enum
+from typing import Any, Dict, List, Literal, Optional, Union
+
+from pydantic import BaseModel, Extra, Field
+
+
+class ErrorModel(BaseModel):
+    """
+    JSON error payload returned in a response with further details on the error
+    """
+
+    message: str = Field(..., description='Human-readable error message')
+    type: str = Field(
+        ...,
+        description='Internal type definition of the error',
+        example='NoSuchNamespaceException',
+    )
+    code: int = Field(
+        ..., description='HTTP response code', example=404, ge=400, le=600
+    )
+    stack: Optional[List[str]] = None
+
+
+class CatalogConfig(BaseModel):
+    """
+    Server-provided configuration for the catalog.
+    """
+
+    overrides: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used to override client configuration; applied after defaults and client configuration.',
+    )
+    defaults: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used as default configuration; applied before client configuration.',
+    )
+
+
+class Updates(BaseModel):
+    pass
+
+
+class UpdateNamespacePropertiesRequest(BaseModel):
+    removals: Optional[List[str]] = Field(
+        None, example=['department', 'access_group'], unique_items=True
+    )
+    updates: Optional[Union[List[str], Updates]] = Field(
+        None, example={'owner': 'Hank Bendickson'}, unique_items=True
+    )
+
+
+class Namespace(BaseModel):
+    """
+    Reference to one or more levels of a namespace
+    """
+
+    __root__: List[str] = Field(
+        ...,
+        description='Reference to one or more levels of a namespace',
+        example=['accounting', 'tax'],
+    )
+
+
+class TableIdentifier(BaseModel):
+    namespace: Namespace
+    name: str
+
+
+class PrimitiveType(BaseModel):
+    __root__: str = Field(..., example=['long', 'string', 'fixed[16]', 'decimal(10,2)'])
+
+
+class Transform(BaseModel):
+    __root__: str = Field(
+        ...,
+        example=[
+            'identity',
+            'year',
+            'month',
+            'day',
+            'hour',
+            'bucket[256]',
+            'truncate[16]',
+        ],
+    )
+
+
+class PartitionField(BaseModel):
+    field_id: Optional[int] = Field(None, alias='field-id')

Review Comment:
   I know python doesn't leave us the option here to use `field-id` as the name, but does `alias` here mean we'd allow either `field-id` _or_ `field_id` in a response? Are there any implications to not strictly requiring `field-id`? cc: @kbendick 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#discussion_r881892059


##########
python/src/iceberg/openapi/rest_catalog.py:
##########
@@ -0,0 +1,503 @@
+# 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.
+# generated by datamodel-codegen:
+#   filename:  rest-catalog-open-api.yaml
+
+from __future__ import annotations
+
+from enum import Enum
+from typing import Any, Dict, List, Literal, Optional, Union
+
+from pydantic import BaseModel, Extra, Field
+
+
+class ErrorModel(BaseModel):
+    """
+    JSON error payload returned in a response with further details on the error
+    """
+
+    message: str = Field(..., description='Human-readable error message')
+    type: str = Field(
+        ...,
+        description='Internal type definition of the error',
+        example='NoSuchNamespaceException',
+    )
+    code: int = Field(
+        ..., description='HTTP response code', example=404, ge=400, le=600
+    )
+    stack: Optional[List[str]] = None
+
+
+class CatalogConfig(BaseModel):
+    """
+    Server-provided configuration for the catalog.
+    """
+
+    overrides: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used to override client configuration; applied after defaults and client configuration.',
+    )
+    defaults: Dict[str, Any] = Field(
+        ...,
+        description='Properties that should be used as default configuration; applied before client configuration.',
+    )
+
+
+class Updates(BaseModel):
+    pass
+
+
+class UpdateNamespacePropertiesRequest(BaseModel):
+    removals: Optional[List[str]] = Field(
+        None, example=['department', 'access_group'], unique_items=True
+    )
+    updates: Optional[Union[List[str], Updates]] = Field(
+        None, example={'owner': 'Hank Bendickson'}, unique_items=True
+    )
+
+
+class Namespace(BaseModel):
+    """
+    Reference to one or more levels of a namespace
+    """
+
+    __root__: List[str] = Field(
+        ...,
+        description='Reference to one or more levels of a namespace',
+        example=['accounting', 'tax'],
+    )
+
+
+class TableIdentifier(BaseModel):
+    namespace: Namespace
+    name: str
+
+
+class PrimitiveType(BaseModel):
+    __root__: str = Field(..., example=['long', 'string', 'fixed[16]', 'decimal(10,2)'])
+
+
+class Transform(BaseModel):
+    __root__: str = Field(
+        ...,
+        example=[
+            'identity',
+            'year',
+            'month',
+            'day',
+            'hour',
+            'bucket[256]',
+            'truncate[16]',
+        ],
+    )
+
+
+class PartitionField(BaseModel):
+    field_id: Optional[int] = Field(None, alias='field-id')

Review Comment:
   You can't use a `-` in a variable in Python, therefore it uses the alias. When going back and forth between dicts and json you can explicitly tell it to use the alias:
   ```python
   ➜  iceberg git:(fd-open-api) ✗ python3
   Python 3.9.13 (main, May 19 2022, 13:48:47) 
   [Clang 13.1.6 (clang-1316.0.21.2)] on darwin
   Type "help", "copyright", "credits" or "license" for more information.
   >>> from pydantic import BaseModel, Field
   >>> 
   >>> class BarModel(BaseModel):
   ...     what_ever: int = Field(alias='what-ever')
   ... 
   >>> BarModel(**{'what-ever': 123})
   BarModel(what_ever=123)
   >>> model = BarModel(**{'what-ever': 123})
   >>> model.dict()
   {'what_ever': 123}
   >>> model.dict(by_alias=True)
   {'what-ever': 123}
   >>> model.json(by_alias=True)
   '{"what-ever": 123}'
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1137539332

   @samredai indeed, that's missing. Thanks for pointing out! 👍🏻 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1140056996

   > Is this worth it?
   
   I think it is! Allow me to elaborate on why I think it is the case.
   
   > Many of the classes in the REST spec mirror structures from TableMetadata, but when we talked last I thought that we were going to create to_dict / from_dict methods to serialize and deserialize those. Having autogenerated classes for those seems like duplication.
   
   That's correct. Having the serialization methods would be replaced by this PR because we get them for free:
   
   ```python
   ➜  python git:(fd-open-api) python3
   Python 3.9.13 (main, May 19 2022, 13:48:47) 
   [Clang 13.1.6 (clang-1316.0.21.2)] on darwin
   Type "help", "copyright", "credits" or "license" for more information.
   >>> from iceberg.openapi.rest_catalog import PartitionSpec, PartitionField
   >>> spec = PartitionSpec(
   ...   spec_id=123,
   ...   fields=[
   ...     PartitionField(
   ...       field_id=234,
   ...       source_id=1,
   ...       name='date',
   ...       transform='identity'
   ...     )
   ...   ]
   ... )
   >>> spec
   PartitionSpec(spec_id=123, fields=[PartitionField(field_id=234, source_id=1, name='date', transform=Transform(__root__='identity'))])
   
   # Serialize
   >>> spec.json()
   '{"spec_id": 123, "fields": [{"field_id": 234, "source_id": 1, "name": "date", "transform": "identity"}]}'
   
   # Deserialize
   >>> PartitionSpec.parse_raw(spec.json())
   PartitionSpec(spec_id=123, fields=[PartitionField(field_id=234, source_id=1, name='date', transform=Transform(__root__='identity'))])
   ```
   
   I think this is much less error-prone as code like this: https://github.com/apache/iceberg/pull/3677/files#diff-9d9a8492ccd85cbbddfc202b9a954b57a079f2cca33e96eb0a9106cf1a4a8130R37-R58
   
   Next to that, the maintenance cost is much higher when we have to keep it in sync by hand. If we generate it, we can check if we break compatibility (using mypy).
   
   > Also, I don't think we're getting much out of this. This makes classes that are simple and could easily be statically defined. And it requires an additional library that looks similar to what we get from @dataclass. I'd probably opt not to pull in the additional dependency.
   
   This is correct, but we get some more on top of the data classes; validators. Instead of using a builder pattern, we can just nicely validate the input using decorators: https://pydantic-docs.helpmanual.io/usage/validators/
   
   This looks like:
   
   ```python
   >>> from iceberg.openapi.rest_catalog import PartitionSpec, PartitionField
   >>> spec = PartitionSpec(
   ...   spec_id=123,
   ...   fields=[
   ...     PartitionField(
   ...       field_id=234,
   ...       name='date',
   ...       transform='identity'
   ...     )
   ...   ]
   ... )
   Traceback (most recent call last):
     File "<stdin>", line 4, in <module>
     File "pydantic/main.py", line 341, in pydantic.main.BaseModel.__init__
   pydantic.error_wrappers.ValidationError: 1 validation error for PartitionField
   source-id
     field required (type=value_error.missing)
   ```
   
   > I'm generally unsure about using the OpenAPI doc currently for generating classes, as it's presently intended more for documentation purposes than for usage for models and generating code. To that extent, it seems that the generated classes are at times not that helpful (like Updates).
   
   I see your point, but it is a bit of a catch-22. Until we start using it, it won't mature. Instead of appending methods to the python classes, it is just as simple as amending the OpenAPI spec and regenerating the code.
   
   > I've also noticed a lot of variance in the generated code depending on which library is used to do it. I know one Python generation tool I used to generate code made all models just dictionaries... not even classes that were secretly dictionaries but just... dictionaries.
   
   Yes, there is a variety of generators and I found this one the best. It nicely generates all the aliases as well. I already bumped into some issues with the plain python one: https://github.com/openapi-generators/openapi-python-client/pull/618
   
   > If we get to a place where the OpenAPI doc can be used as a drop-in, that would be great. But talking to some people who are very familiar with OpenAPI, my understanding is that some projects have behavior that's simply more complex than any of the OpenAPI generation tools can really account for. I've been told that either the code has to be written in a very specific style or that it can take years to get to such a place.
   
   Great point, and I think we always need to extend the generated classes with convenience methods, or additional validation. How to do this is given in https://github.com/apache/iceberg/pull/4717#discussion_r882941190
   
   > Is pydantic required if we were to move forward with this or is this just added in this PR for other reasons?
   
   For this implementation, pydantic is required: https://github.com/koxudaxi/datamodel-code-generator The code generator specifically generates the pydantic object (and brings in the validation as well). It fits our use-case perfectly: https://python.plainenglish.io/an-introduction-to-the-pydantic-stack-9e490d606c8d
   
   It is coming from the FastAPI movement, and it is becoming increasingly more popular. Also, pydantic doesn't pull in the whole universe, it just has one dependency: https://github.com/samuelcolvin/pydantic/blob/master/setup.py#L131-L133


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1140487797

   Alright, did some more in-depth investigation over the weekend by just converting everything to the generated models. It went quite well and it works quite nicely with the encoders/decoders. We're able to export the schema from our subclassed types by taking the parent schema of the pydantic model, which is quite cool. 
   
   But hit a wall now. At the top we have the `Type`:
   
   https://github.com/apache/iceberg/blob/f5de18f554711b4ae00d6ebefbaa126a7f83f792/open-api/rest-catalog-open-api.yaml#L940-L945
   
   With the generated models, we're unable to construct the type hierarchy because the `PrimitiveType` is a plain `string`, and the other ones are of type `object` 😭 The combination of Pydantic and subclassing isn't able to cope with this. One way would be to change the PrimitiveType also to an `object` (and also add a `type` field to determine the type). This would also allow us to add properties for the Fixed and Decimal instead of having to parse them from the name.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4858:
URL: https://github.com/apache/iceberg/pull/4858#issuecomment-1140529276

   Unfortunately, we can't change the primitive type to an object. That serialization is part of the table spec so we can't update the representation. We could possibly change the OpenAPI representation, though. It didn't make sense to me that the generator was expecting `Type` branches to be uniform, so I looked at the [`anyOf` docs](https://swagger.io/docs/specification/data-models/oneof-anyof-allof-not/). Looks like `anyOf` can be one or more branches, which implies that there is some similarity across branches. However, `oneOf` expects the type to match exactly one branch. Maybe that would fix this issue?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko closed pull request #4858: Python: Add generated classes from OpenAPI spec

Posted by GitBox <gi...@apache.org>.
Fokko closed pull request #4858: Python: Add generated classes from OpenAPI spec
URL: https://github.com/apache/iceberg/pull/4858


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org