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/09/27 17:13:42 UTC

[GitHub] [iceberg] JonasJ-ap opened a new pull request, #5870: Python: Add GlueCatalog Support

JonasJ-ap opened a new pull request, #5870:
URL: https://github.com/apache/iceberg/pull/5870

   Under development


-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"

Review Comment:
   the proper property to use is https://github.com/apache/iceberg/blob/master/python/pyiceberg/catalog/hive.py#L111



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyproject.toml:
##########
@@ -63,6 +63,8 @@ python-snappy = { version = "^0.6.1", optional = true }
 thrift = { version = "^0.16.0", optional = true }
 
 s3fs = { version = "2022.8.2", optional = true }
+boto3 = {version = "1.24.59", optional = true}

Review Comment:
   nit: space after true



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996362768


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)

Review Comment:
   Thank you for your suggestion. I refactored it to take advantage of the `load_table` below



-- 
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 closed pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
rdblue closed pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support
URL: https://github.com/apache/iceberg/pull/5870


-- 
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 #5870: Python: Add GlueCatalog Table Creation and Loading Support

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

   Closing this in favor of #6034.


-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"

Review Comment:
   `{table_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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': properties[PROP_TABLE_DESCRIPTION]
+                    if properties and PROP_TABLE_DESCRIPTION in properties else "",
+                    'TableType': EXTERNAL_TABLE_TYPE,
+                    'Parameters': _construct_parameters(metadata_location),
+                }
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        loaded_table = load_table_response[PROP_GLUE_TABLE]
+        io = load_file_io(
+            {**self.properties},

Review Comment:
   I think we don't need the `{**}` syntax? since there is only 1 properties



-- 
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] jackye1995 commented on pull request #5870: Python: Add GlueCatalog Support

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

   @Fokko I was just planning to discuss with you about your thoughts on this. As we see this will be overall very similar to the HiveCatalog, just like the java implementation. Do you think it is worth having a base catalog class, or whatever more pythonic way to handle the situation?


-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984955025


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed

Review Comment:
   Thank you for your suggestions. Just to confirm, here the `description` should be one of the props in `properties`. So I think maybe  I can directly extract description from the `properties`?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:

Review Comment:
   yes this is fine. We can be more specific in the general comment, like `Create an Iceberg table in Glue catalog`



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/tests/catalog/test_glue.py:
##########
@@ -0,0 +1,58 @@
+#  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.
+from unittest.mock import patch, MagicMock
+
+import botocore.exceptions
+
+from pyiceberg.catalog import PropertiesUpdateSummary
+from pyiceberg.catalog.glue import GlueCatalog
+from pyiceberg.schema import Schema
+
+import random
+import string
+
+
+def get_randam_table_name():

Review Comment:
   sound good, we should definitely keep the files around, because ideally we want to also do integration tests like what we did in Java.



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984993664


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': properties[PROP_TABLE_DESCRIPTION]

Review Comment:
   Ah, I see. Thank you for your suggestion. Since the `Description` field does not accept `None` directly, I made a helper method called `_construct_table_input` to implement the logic here. I think the added method can make the code structure clear when we want to add support for `StorageDescriptor` in the future.



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': properties[PROP_TABLE_DESCRIPTION]

Review Comment:
   Ah, I see. Thank you for your suggestion. Since the `Description` field does not accept `None` directly, I made a helper method called `_construct_table_input` to implement the logic here. I think the added method can also make the code structure clear when we want to add support for `StorageDescriptor` in the future.



-- 
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 #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}

Review Comment:
   I would collapse this into a single line and return right away



##########
python/pyproject.toml:
##########
@@ -63,6 +63,7 @@ python-snappy = { version = "^0.6.1", optional = true }
 thrift = { version = "^0.16.0", optional = true }
 
 s3fs = { version = "2022.8.2", optional = true }
+boto3 = "1.24.59"

Review Comment:
   Could you make this one optional? Similar to Hive. Because not everyone uses Glue or Boto, this makes the package quite heavy to install.



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:

Review Comment:
   ```suggestion
   def _construct_parameters(metadata_location: str) -> Properties:
   ```



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"

Review Comment:
   ```suggestion
   PROP_GLUE_TABLE_TYPE = "TableType"
   ```



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Dict[str, str]) -> Dict[str, Any]:

Review Comment:
   ```suggestion
   def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Properties:
   ```



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Dict[str, str]) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        Prop_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if properties and PROP_GLUE_TABLE_DESCRIPTION in properties:
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = properties[PROP_GLUE_TABLE_DESCRIPTION]
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]

Review Comment:
   ```suggestion
       properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
   ```



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Dict[str, str]) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        Prop_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if properties and PROP_GLUE_TABLE_DESCRIPTION in properties:

Review Comment:
   You could also leverage the walrus operator here:
   ```suggestion
       if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
   ```



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Dict[str, str]) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        Prop_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if properties and PROP_GLUE_TABLE_DESCRIPTION in properties:
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = properties[PROP_GLUE_TABLE_DESCRIPTION]
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        loaded_table = load_table_response[PROP_GLUE_TABLE]
+        io = load_file_io(self.properties, loaded_table[PROP_GLUE_TABLE_PARAMETERS][PROP_METADATA_LOCATION])
+        return _convert_glue_to_iceberg(loaded_table, io)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        raise NotImplementedError("currently unsupported")
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        raise NotImplementedError("currently unsupported")
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        raise NotImplementedError("currently unsupported")

Review Comment:
   Can you implement this one? This makes it easier to test. Should be something like `self.glue.get_databases()`



##########
python/tests/catalog/test_glue.py:
##########
@@ -0,0 +1,77 @@
+#  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.
+import getpass as gt
+import random
+import string
+
+import pytest
+
+from pyiceberg.catalog.glue import GlueCatalog
+from pyiceberg.exceptions import NoSuchNamespaceError
+from pyiceberg.schema import Schema

Review Comment:
   For testing we could also use moto: https://docs.getmoto.org/en/latest/docs/services/glue.html
   
   What do you think @jackye1995 ?



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,242 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+Prop_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Dict[str, str]) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        Prop_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if properties and PROP_GLUE_TABLE_DESCRIPTION in properties:
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = properties[PROP_GLUE_TABLE_DESCRIPTION]
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:

Review Comment:
   ```suggestion
   def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
   ```



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)
+        except self.client.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        # TODO: may need to add table properties to the io too
+        io = load_file_io(

Review Comment:
   The most important ones are properties and location. I guess that we could also infer the location from the properties, but I don't think that's the cleanest approach, and we should just get it from the location property.



-- 
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] JonasJ-ap commented on pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#issuecomment-1265818424

   > Hey @JonasJ-ap First of all, many thanks for picking this up 🙏🏻 I have many comments, but they are rather small, this is looking great 👍🏻
   
   Thank you very much for your review and suggestions. I'll update the implementation based on them.


-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984953943


##########
python/tests/catalog/test_glue.py:
##########
@@ -0,0 +1,58 @@
+#  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.
+from unittest.mock import patch, MagicMock
+
+import botocore.exceptions
+
+from pyiceberg.catalog import PropertiesUpdateSummary
+from pyiceberg.catalog.glue import GlueCatalog
+from pyiceberg.schema import Schema
+
+import random
+import string
+
+
+def get_randam_table_name():

Review Comment:
   Thank you for your suggestions. The current `test_glue.py` is a placeholder to remind me to add unit tests later. Since `GlueCatalog` is still under development, I will keep the temporal tests for a moment for convenient verification of my changes to the code. I will change them to actual unit tests based on what you suggested once we finish reviewing the `create_table` and `load_table`.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': properties[PROP_TABLE_DESCRIPTION]

Review Comment:
   is description required to create the Glue table? If not we should use None instead of ""



-- 
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] jackye1995 commented on pull request #5870: Python: Add GlueCatalog Support

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

   Also can we update the title of the PR to be just supporting the table creation and loading? Just to be more specific about the purpose of it.


-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties

Review Comment:
   this can just be null



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties

Review Comment:
   this can just be None



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:

Review Comment:
   Case sensitive?



-- 
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] jackye1995 commented on pull request #5870: Python: Add GlueCatalog Support

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

   Thanks for picking this up! I think there are some conceptual level things we need to clarify first. This is a large effort and I would suggest we can start from the very base case. I would suggest making everything else not implemented, and just implement `create_table` and `load_table` so that we get those parts 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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties
+                    else boto3.client("sts").get_caller_identity().get("Account"),
+                    'LastAccessTime': now,

Review Comment:
   no need to set the access and analyzed time



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:

Review Comment:
   please add comments to the method like https://github.com/apache/iceberg/blob/master/python/pyiceberg/catalog/hive.py#L285



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+        Args:
+            identifier: Table identifier.
+        Returns:
+            Table: the table instance with its metadata
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        loaded_table = load_table_response[PROP_GLUE_TABLE]
+        io = load_file_io(self.properties, loaded_table[PROP_GLUE_TABLE_PARAMETERS][PROP_METADATA_LOCATION])

Review Comment:
   IO instantiation looks fishy to me. We need to make sure we're doing it consistently. Might want to handle this in `_convert_glue_to_iceberg`.



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "

Review Comment:
   Nit: embed `PROP_TABLE_TYPE`?



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"

Review Comment:
   Can we share some of these constants with Hive?



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996364578


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,249 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _convert_glue_to_iceberg(self, glue_table: Dict[str, Any]) -> Table:
+        properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+        if PROP_TABLE_TYPE not in properties:
+            raise NoSuchTableError(
+                f"Property {PROP_TABLE_TYPE} missing, could not determine type: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+        glue_table_type = properties[PROP_TABLE_TYPE]
+
+        if glue_table_type.upper() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+
+        if PROP_METADATA_LOCATION not in properties:
+            raise NoSuchTableError(
+                f"Table property {PROP_METADATA_LOCATION} is missing, cannot find metadata for: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+        metadata_location = properties[PROP_METADATA_LOCATION]
+
+        io = load_file_io(properties=self.properties, location=metadata_location)

Review Comment:
   For consistency, as mentioned by @Fokko and @jackye1995 in previous [discussion](https://github.com/apache/iceberg/pull/5870#discussion_r985725785), `properties` and `location` are the primary things here. Hence, I made the two io instantiation using `self.properties` and `metadata_location` to load the IO



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996371682


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"

Review Comment:
   I made a prototype of sharing by creating `pyiceberg.catalog.base` and put `ICEBERG`, `TABLE_TYPE`, `WAREHOUSE`, and `METADATA_LOCATION` in it. Do you think it may be better to create a class to hold these variables instead?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed

Review Comment:
   can take from description in `metadata`



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984947731


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)

Review Comment:
   Oh, I forgot to delete this check after adding the exception caught. Sorry about that. Thank you for pointing this 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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r983093603


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,200 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)

Review Comment:
   Currently, (follow the hive.py's logic) after we create the table, we use `get_table` to fetch the created table from the remote and return it. I wonder if it is ok directly use `metadata`, `metadata_location`, ... above to create the return table and uses caught exception to verify the creation of the table?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")

Review Comment:
   this can be named `glue`



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")

Review Comment:
   Should there be a subclass of `NoSuchTableError` to signal that the table exists, but is not Iceberg? We do that in Java.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,

Review Comment:
   inputs like `'Name'` can also be prop constants



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984951487


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:

Review Comment:
   Currently, I just copy the comment from `hive.py` since the comment just describes the general usage of `Catalog.create_table` and `Catalog.load_table`. Do you think there is some additional information specific to `GlueCatalog` that should be added to the comment?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:

Review Comment:
   similar comment for load_table



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996363442


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+        Args:
+            identifier: Table identifier.
+        Returns:
+            Table: the table instance with its metadata
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        loaded_table = load_table_response[PROP_GLUE_TABLE]
+        io = load_file_io(self.properties, loaded_table[PROP_GLUE_TABLE_PARAMETERS][PROP_METADATA_LOCATION])
+        return _convert_glue_to_iceberg(loaded_table, io)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        raise NotImplementedError("currently unsupported")
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        raise NotImplementedError("currently unsupported")
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+        # Glue does not support hierarchical namespace, therefore return an empty list
+        if namespace:
+            return []
+        databases_response = self.glue.get_databases()
+        return [
+            self.identifier_to_tuple(database[PROP_GLUE_DATABASE_NAME])

Review Comment:
   Thank you for your review. The names coming back from Glue are `string`. According to the interface definition: https://github.com/apache/iceberg/blob/336d18acdb7dc0e783861151e2e871524cef9f6d/python/pyiceberg/catalog/__init__.py#L258, we need to return a list of `Identifier`, which is a tuple defined as :https://github.com/apache/iceberg/blob/336d18acdb7dc0e783861151e2e871524cef9f6d/python/pyiceberg/typedef.py#L36
   Hence, I use `identifier_to_tuple` to convert the names to `Identifer` so that they can meet the interface requirement. If I misunderstand some points, please let me know. 



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)

Review Comment:
   Can this call `load_table` instead of duplicating code?



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r990732533


##########
python/tests/catalog/test_glue.py:
##########
@@ -0,0 +1,121 @@
+#  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.
+import getpass as gt
+import random
+import string
+
+import pytest
+from moto import mock_glue, mock_s3
+
+from pyiceberg.catalog.glue import GlueCatalog
+from pyiceberg.exceptions import NoSuchNamespaceError
+from pyiceberg.schema import Schema
+
+# early develop stage only, change this to a user with aws cli configured locally
+MY_USERNAME = "jonasjiang"
+
+
+def get_random_table_name():
+    prefix = "my_iceberg_table-"
+    random_tag = "".join(random.choice(string.ascii_letters) for _ in range(20))
+    return (prefix + random_tag).lower()
+
+
+@pytest.mark.skipif(gt.getuser() != MY_USERNAME, reason="currently need aws account, will be unit test later")
+def test_create_table(table_schema_nested: Schema):
+    table_name = get_random_table_name()
+    identifier = ("myicebergtest", table_name)
+    table = GlueCatalog("glue").create_table(
+        identifier, table_schema_nested, f"s3://pythongluetest/myicebergtest.db/{table_name}"
+    )
+    assert table.identifier == identifier
+
+
+@pytest.mark.skipif(gt.getuser() != MY_USERNAME, reason="currently need aws account, will be unit test later")
+def test_create_table_with_default_location(table_schema_nested: Schema):
+    table_name = get_random_table_name()
+    identifier = ("myicebergtest", table_name)
+    test_catalog = GlueCatalog("glue", warehouse="s3://pythongluetest")
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+
+
+@pytest.mark.skipif(gt.getuser() != MY_USERNAME, reason="currently need aws account, will be unit test later")
+def test_create_table_with_invalid_database(table_schema_nested: Schema):
+    table_name = get_random_table_name()
+    identifier = ("invalid", table_name)
+    test_catalog = GlueCatalog("glue", warehouse="s3://pythongluetest")
+    with pytest.raises(NoSuchNamespaceError):
+        test_catalog.create_table(identifier, table_schema_nested)
+
+
+@pytest.mark.skipif(gt.getuser() != MY_USERNAME, reason="currently need aws account, will be unit test later")
+def test_create_table_with_invalid_location(table_schema_nested: Schema):
+    table_name = get_random_table_name()
+    identifier = ("myicebergtest", table_name)
+    test_catalog = GlueCatalog("glue")
+    with pytest.raises(ValueError):
+        test_catalog.create_table(identifier, table_schema_nested)
+
+
+@pytest.mark.skipif(gt.getuser() != MY_USERNAME, reason="currently need aws account, will be unit test later")
+def test_load_table():
+    table = GlueCatalog("glue").load_table(("myicebergtest", "loadtest"))
+    assert table.identifier == ("myicebergtest", "loadtest")
+
+
+@pytest.mark.skipif(gt.getuser() != MY_USERNAME, reason="currently need aws account, will be unit test later")
+def test_list_namespaces():
+    db_list = GlueCatalog("glue").list_namespaces()
+    assert db_list == [("listdatabasetest",), ("myicebergtest",)]
+
+
+# prototype of unit test
+@mock_s3
+@mock_glue
+def test_unit_create_table(_s3, _glue, _patch_aiobotocore, table_schema_nested):

Review Comment:
   Added two unit test prototypes using moto and pytest fixture



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties
+                    else boto3.client("sts").get_caller_identity().get("Account"),
+                    'LastAccessTime': now,
+                    'LastAnalyzedTime': now,
+                    'StorageDescriptor': {
+                        'Columns': list(map(_convert_column, schema.fields)),
+                        'Location': location or 's3://',  # To be fixed
+                        'BucketColumns': [
+                            'string',
+                        ],
+                        'SortColumns': [{
+                            schema.find_column_name(field.source_id),
+                            1 if field.direction == SortDirection.ASC else 0
+                        } for field in sort_order.fields]
+                    },
+                    'PartitionKeys': [

Review Comment:
   same here, we can skip partition keys for the initial implementation



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996364503


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+        Args:
+            identifier: Table identifier.
+        Returns:
+            Table: the table instance with its metadata
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        loaded_table = load_table_response[PROP_GLUE_TABLE]
+        io = load_file_io(self.properties, loaded_table[PROP_GLUE_TABLE_PARAMETERS][PROP_METADATA_LOCATION])

Review Comment:
   Thank you for your suggestion. I refactored the code to initialize the IO inside `_convert_glue_to_iceberg`. 
   
   For consistency, I made the two io instantiations using `self.properties` and `metadata_location` to load the IO:
   ```
   io = load_file_io(properties=self.properties, location=metadata_location)
   ```
   
   



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996368485


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")

Review Comment:
   Thank you for your suggestion. I added an exception class called `NoSuchIcebergTableError` (similar to that in Java) and raised it when table type is not `iceberg`.
   
   I also find another exception called:
   https://github.com/apache/iceberg/blob/336d18acdb7dc0e783861151e2e871524cef9f6d/python/pyiceberg/exceptions.py#L79
   which seems more suitable when a property is missing (like what happened here). Do you think it is a good idea to raise this exception here?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)
+        except self.client.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        # TODO: may need to add table properties to the io too
+        io = load_file_io(

Review Comment:
   this should not need table properties. Also note that there are 2 things here we should not confuse, what you get from `load_table_response['Table']['Parameters']` is Glue's table parameters. That is different from Iceberg table properties, those are stored in S3 in the table metadata file.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)
+        except self.client.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        # TODO: may need to add table properties to the io too
+        io = load_file_io(
+            {**self.properties, **load_table_response['Table']['Parameters']},
+            load_table_response['Table']['StorageDescriptor']['Location'])

Review Comment:
   similar to the comment above, `load_table_response['Table']['StorageDescriptor']['Location']` is not the source of truth. Here location is used to infer FileIO implementation, if we really want we should use the `metadata_location` path.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)
+        except self.client.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        # TODO: may need to add table properties to the io too
+        io = load_file_io(

Review Comment:
   Yes. I see in `hive.py` the implementation is `io = load_file_io({**self.properties, **hive_table.parameters}, hive_table.sd.location)`. Maybe this is to allow users to put `py-io-impl` as a table parameter, but that still feels like a strange experience to me. I guess @Fokko can comment more about why it's done that way in Hive.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,246 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+PROP_TABLE_DESCRIPTION = "description"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Dict[str, str]:
+    properties = {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+    return properties
+
+
+def _convert_glue_to_iceberg(glue_table, io: FileIO) -> Table:
+    properties: Dict[str, str] = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}")
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist")
+
+        if PROP_GLUE_DATABASE_LOCATION in response[PROP_GLUE_DATABASE]:
+            return f"{response[PROP_GLUE_DATABASE][PROP_GLUE_DATABASE]}/table_name"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    # tested on pre-existing database

Review Comment:
   nit: remove comment



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+        Args:
+            identifier: Table identifier.
+        Returns:
+            Table: the table instance with its metadata
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        loaded_table = load_table_response[PROP_GLUE_TABLE]
+        io = load_file_io(self.properties, loaded_table[PROP_GLUE_TABLE_PARAMETERS][PROP_METADATA_LOCATION])
+        return _convert_glue_to_iceberg(loaded_table, io)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        raise NotImplementedError("currently unsupported")
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        raise NotImplementedError("currently unsupported")
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        raise NotImplementedError("currently unsupported")
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+        # Glue does not support hierarchical namespace, therefore return an empty list
+        if namespace:
+            return []
+        databases_response = self.glue.get_databases()
+        return [
+            self.identifier_to_tuple(database[PROP_GLUE_DATABASE_NAME])

Review Comment:
   Why does this need to parse a name coming back from Glue?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")

Review Comment:
   Looks like we should also do that for hive.py: https://github.com/apache/iceberg/blob/master/python/pyiceberg/catalog/hive.py#L245-L254



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996364590


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,249 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _convert_glue_to_iceberg(self, glue_table: Dict[str, Any]) -> Table:
+        properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+        if PROP_TABLE_TYPE not in properties:
+            raise NoSuchTableError(
+                f"Property {PROP_TABLE_TYPE} missing, could not determine type: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+        glue_table_type = properties[PROP_TABLE_TYPE]
+
+        if glue_table_type.upper() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+
+        if PROP_METADATA_LOCATION not in properties:
+            raise NoSuchTableError(
+                f"Table property {PROP_METADATA_LOCATION} is missing, cannot find metadata for: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+        metadata_location = properties[PROP_METADATA_LOCATION]
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+            metadata=metadata,
+            metadata_location=metadata_location,
+        )
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)

Review Comment:
   Same as above



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996364578


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,249 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _convert_glue_to_iceberg(self, glue_table: Dict[str, Any]) -> Table:
+        properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+        if PROP_TABLE_TYPE not in properties:
+            raise NoSuchTableError(
+                f"Property {PROP_TABLE_TYPE} missing, could not determine type: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+        glue_table_type = properties[PROP_TABLE_TYPE]
+
+        if glue_table_type.upper() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+
+        if PROP_METADATA_LOCATION not in properties:
+            raise NoSuchTableError(
+                f"Table property {PROP_METADATA_LOCATION} is missing, cannot find metadata for: "
+                f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+            )
+        metadata_location = properties[PROP_METADATA_LOCATION]
+
+        io = load_file_io(properties=self.properties, location=metadata_location)

Review Comment:
   For consistency, as mentioned by @Fokko and @jackye1995 in previous [discussion](https://github.com/apache/iceberg/pull/5870#discussion_r985725785), `properties` and `location` are the most important here. Hence, I made the two io instantiation using `self.properties` and `metadata_location` to load the IO



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996362631


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:

Review Comment:
   Thank you for your suggestion. Since I also use `ICEBERG` to set the table parameters in `_construct_table_input`, I will make everything in uppercase to make the check case insensitive.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"

Review Comment:
   +1, not sure what's the best way though, should it be in some shared module like `pyiceberg.catalog.base`? Any suggestions?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties
+                    else boto3.client("sts").get_caller_identity().get("Account"),
+                    'LastAccessTime': now,
+                    'LastAnalyzedTime': now,
+                    'StorageDescriptor': {
+                        'Columns': list(map(_convert_column, schema.fields)),
+                        'Location': location or 's3://',  # To be fixed
+                        'BucketColumns': [
+                            'string',
+                        ],
+                        'SortColumns': [{
+                            schema.find_column_name(field.source_id),
+                            1 if field.direction == SortDirection.ASC else 0
+                        } for field in sort_order.fields]
+                    },
+                    'PartitionKeys': [
+                        {
+                            'Name': schema.find_column_name(spec.source_id),
+                            'Type': str(schema.find_type(spec.source_id)),
+                            'Comment': str(spec.transform)
+                        }
+                        for spec in partition_spec.fields],
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': properties,

Review Comment:
   the code logic here is not quite right. I think we have a misunderstanding of how GlueCatalog works with Iceberg tables. The necessary information stored in Glue is just table name, description, and parameters `table_type=ICEBERG` and `metadata_location=s3://xxxxx`. Everything else are written to S3 as a part of Iceberg base library here https://github.com/apache/iceberg/blob/master/python/pyiceberg/catalog/hive.py#L307-L312. I would suggest read `hive.py` to see how that works and then map to what should be implemented here.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties
+                    else boto3.client("sts").get_caller_identity().get("Account"),
+                    'LastAccessTime': now,
+                    'LastAnalyzedTime': now,
+                    'StorageDescriptor': {

Review Comment:
   we can skip all the contents in the storage descriptor for the initial implementation



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984951487


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:

Review Comment:
   Currently, I just copy the comment from `hive.py` since the comment just describes the general usage of `Catalog.create_table` and `Catalog.create_table`. Do you think there is some additional information specific to `GlueCatalog` that should be added to the comment?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/tests/catalog/test_glue.py:
##########
@@ -0,0 +1,58 @@
+#  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.
+from unittest.mock import patch, MagicMock
+
+import botocore.exceptions
+
+from pyiceberg.catalog import PropertiesUpdateSummary
+from pyiceberg.catalog.glue import GlueCatalog
+from pyiceberg.schema import Schema
+
+import random
+import string
+
+
+def get_randam_table_name():

Review Comment:
   for testing I think we need to use some mocks so it's not contacting actual AWS service and can be run as unit test



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,200 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)

Review Comment:
   I think it's beneficial to fetch the created result, some information are filled during the creation process
   



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")

Review Comment:
   looks like this is not used?



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r983099664


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,237 @@
+#  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.
+from __future__ import annotations
+
+import getpass
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.schema import Schema
+from pyiceberg.table import Table
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import Identifier, Properties, EMPTY_DICT
+from pyiceberg.types import NestedField
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, properties: Properties):
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+        super().__init__(name, **properties)
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table):
+        # TODO
+        pass
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: str | Identifier,
+            schema: Schema,
+            location: str | None = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        now = datetime.now()
+
+        def _convert_column(field: NestedField):
+            d = {'Name': field.name, 'Type': str(field.field_type)}
+
+            if field.doc:
+                d['Comment'] = field.doc
+
+            return d
+
+        # Do all the metadata foo once the Hive PR has been merged
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': 'string',  # To be fixed
+                    'Owner': properties[OWNER] if properties and OWNER in properties
+                    else boto3.client("sts").get_caller_identity().get("Account"),
+                    'LastAccessTime': now,
+                    'LastAnalyzedTime': now,
+                    'StorageDescriptor': {
+                        'Columns': list(map(_convert_column, schema.fields)),
+                        'Location': location or 's3://',  # To be fixed
+                        'BucketColumns': [
+                            'string',
+                        ],
+                        'SortColumns': [{
+                            schema.find_column_name(field.source_id),
+                            1 if field.direction == SortDirection.ASC else 0
+                        } for field in sort_order.fields]
+                    },
+                    'PartitionKeys': [
+                        {
+                            'Name': schema.find_column_name(spec.source_id),
+                            'Type': str(schema.find_type(spec.source_id)),
+                            'Comment': str(spec.transform)
+                        }
+                        for spec in partition_spec.fields],
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': properties,

Review Comment:
   Thank you for your suggestions. I'll redesign my implementation following the logic in `hive.py`



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984949646


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)
+        except self.client.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e
+        # TODO: may need to add table properties to the io too
+        io = load_file_io(

Review Comment:
   Thank you for your clarification. Just to confirm, if we want to set the prop `py-io-impl` for the `FileIO`, we should make it a property of `GlueCatalog`, 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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r984950379


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}

Review Comment:
   Thank you for your suggestion. I refactored the code so that all constants are defined at the beginning.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:

Review Comment:
   yeah this should be case insensitive



-- 
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 a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:
+        raise NoSuchTableError(
+            f"Property table_type is {glue_table_type}, expected {ICEBERG}: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    if prop_meta_location := properties.get(PROP_METADATA_LOCATION):
+        metadata_location = prop_meta_location
+    else:
+        raise NoSuchTableError(f"Table property {PROP_METADATA_LOCATION} is missing")
+
+    file = io.new_input(metadata_location)
+    metadata = FromInputFile.table_metadata(file)
+    return Table(
+        identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]),
+        metadata=metadata,
+        metadata_location=metadata_location,
+    )
+
+
+def _write_metadata(metadata: TableMetadata, io: FileIO, metadate_path: str):
+    ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+
+class GlueCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.glue = boto3.client("glue")
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.glue.get_database(Name=database_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"The database: {database_name} does not exist") from e
+
+        if database_location := response.get(PROP_GLUE_DATABASE).get(PROP_GLUE_DATABASE_LOCATION):
+            return f"{database_location}/{table_name}"
+
+        if PROP_WAREHOUSE in self.properties:
+            return f"{self.properties[PROP_WAREHOUSE]}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """Create an Iceberg table in Glue catalog
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+        Returns:
+            Table: the created table instance
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        _write_metadata(metadata, io, metadata_location)
+        try:
+            self.glue.create_table(
+                DatabaseName=database_name, TableInput=_construct_table_input(table_name, metadata_location, properties)
+            )
+        except self.glue.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} not found") from e
+
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table {database_name}.{table_name} fail to be created") from e
+
+        glue_table = load_table_response[PROP_GLUE_TABLE]
+        return _convert_glue_to_iceberg(glue_table, io)
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """Loads the table's metadata and returns the table instance.
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+        Args:
+            identifier: Table identifier.
+        Returns:
+            Table: the table instance with its metadata
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
+        except self.glue.exceptions.EntityNotFoundException as e:
+            raise NoSuchTableError(f"Table does not exists: {table_name}") from e

Review Comment:
   Missing `database_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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}

Review Comment:
   Constants that should be defined similar to https://github.com/apache/iceberg/blob/master/python/pyiceberg/catalog/hive.py#L147. I know there are a few duplicates, once we have those in place we can then discuss what could be extracted to a base.



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)

Review Comment:
   are you sure we need this check? shouldn't the aws client just fail and throw error if this fails?



##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog
+        metadata_location = f"{location}/metadata/{uuid.uuid4()}.metadata.json"
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order,
+            properties=properties
+        )
+        io = load_file_io({**self.properties, **properties}, location=location)
+        self._write_metadata(metadata, io, metadata_location)
+        try:
+            create_table_response = self.client.create_table(
+                DatabaseName=database_name,
+                TableInput={
+                    'Name': table_name,
+                    'Description': '',  # To be fixed
+                    'TableType': 'EXTERNAL_TABLE',
+                    'Parameters': self._construct_parameters(metadata_location),
+                }
+            )
+            # TODO: check response
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            glue_table = load_table_response['Table']
+        except self.client.exceptions.AlreadyExistsException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        return self._glue_to_iceberg(glue_table, io)
+
+    # tested
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+        try:
+            load_table_response = self.client.get_table(DatabaseName=database_name, Name=table_name)
+            self._check_response(load_table_response)

Review Comment:
   are you sure we need this check? shouldn't the aws client just throw error if this fails?



-- 
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] jackye1995 commented on a diff in pull request #5870: Python: Add GlueCatalog Support

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,202 @@
+#  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.
+
+
+import getpass
+import uuid
+
+import boto3
+from datetime import datetime
+from typing import Union, Optional, List, Set, Dict
+
+from pyiceberg.catalog.hive import OWNER
+
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+)
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import PartitionSpec, UNPARTITIONED_PARTITION_SPEC
+from pyiceberg.table.sorting import SortOrder, UNSORTED_SORT_ORDER, SortDirection
+from pyiceberg.typedef import EMPTY_DICT
+
+from pyiceberg.types import NestedField
+
+METADATA_LOCATION = "metadata_location"
+ICEBERG = "iceberg"
+
+
+class GlueCatalog(Catalog):
+
+    def __init__(self, name: str, **properties: Properties):
+        super().__init__(name, **properties)
+        self.client = boto3.client("glue")
+        self.sts_client = boto3.client("sts")
+
+
+    def _check_response(self, response: Dict[str, Dict[str, str]]):
+        if response['ResponseMetadata']['HTTPStatusCode'] != 200:
+            raise ValueError(f"Got unexpected status code {response['HttpStatusCode']}")
+
+    def _glue_to_iceberg(self, glue_table, io: FileIO) -> Table:
+        properties: Dict[str, str] = glue_table["Parameters"]
+
+        if "table_type" not in properties:
+            raise NoSuchTableError(
+                f"Property table_type missing, could not determine type: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        glue_table_type = properties.get("table_type")
+        if glue_table_type.lower() != ICEBERG:
+            raise NoSuchTableError(
+                f"Property table_type is {glue_table_type}, expected {ICEBERG}: {glue_table['DatabaseName']}.{glue_table['Name']}")
+        if prop_meta_location := properties.get(METADATA_LOCATION):
+            metadata_location = prop_meta_location
+        else:
+            raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing")
+
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(glue_table['DatabaseName'], glue_table['Name']),
+            metadata=metadata,
+            metadata_location=metadata_location
+        )
+
+    def _iceberg_to_glue(self, iceberg_table):
+        # TODO
+        pass
+
+    def _construct_parameters(self, metadata_location: str) -> Dict[str, str]:
+        properties = {"table_type": "ICEBERG", "metadata_location": metadata_location}
+        return properties
+
+    def _default_warehouse_location(self, database_name: str, table_name: str):
+        try:
+            response = self.client.get_database(Name=database_name)
+        # TODO: handle response and errors
+        except:
+            raise NoSuchNamespaceError("Database not found")
+
+        if "LocationUri" in response["Database"]:
+            return f"{response['Database']['LocationUri']}/table_name"
+
+        # TODO: should extract warehouse path from the properties and handle potential errors
+        return f"{self.properties['warehouse_path']}/{database_name}.db/{table_name}"
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str):
+        if not location:
+            return self._default_warehouse_location(database_name, table_name)
+        return location
+
+    def _write_metadata(self, metadata: TableMetadata, io: FileIO, metadate_path: str):
+        ToOutputFile.table_metadata(metadata, io.new_output(metadate_path))
+
+    # tested on pre-existing database
+    def create_table(
+            self,
+            identifier: Union[str, Identifier],
+            schema: Schema,
+            location: Optional[str] = None,
+            partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+            sort_order: SortOrder = UNSORTED_SORT_ORDER,
+            properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        database_name, table_name = self.identifier_to_tuple(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        # TODO: give it correct path based on java version of glueCatalog

Review Comment:
   can follow https://github.com/apache/iceberg/blob/master/python/pyiceberg/catalog/hive.py#L307



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996362631


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:

Review Comment:
   Thank you for your suggestion. Since I also use `ICEBERG` to set the table parameters in `_construct_table_parameters`, I will make everything in uppercase to make the check case insensitive.



-- 
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] JonasJ-ap commented on a diff in pull request #5870: Python: Add GlueCatalog Table Creation and Loading Support

Posted by GitBox <gi...@apache.org>.
JonasJ-ap commented on code in PR #5870:
URL: https://github.com/apache/iceberg/pull/5870#discussion_r996362631


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -0,0 +1,247 @@
+#  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.
+
+
+import uuid
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError, TableAlreadyExistsError
+from pyiceberg.io import FileIO, load_file_io
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile, ToOutputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import TableMetadata, new_table_metadata
+from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+ICEBERG = "ICEBERG"
+EXTERNAL_TABLE_TYPE = "EXTERNAL_TABLE"
+
+PROP_TABLE_TYPE = "table_type"
+PROP_WAREHOUSE = "warehouse"
+PROP_METADATA_LOCATION = "metadata_location"
+
+PROP_GLUE_TABLE = "Table"
+PROP_GLUE_TABLE_TYPE = "TableType"
+PROP_GLUE_TABLE_DESCRIPTION = "description"
+PROP_GLUE_TABLE_PARAMETERS = "Parameters"
+PROP_GLUE_TABLE_DATABASE_NAME = "DatabaseName"
+PROP_GLUE_TABLE_NAME = "Name"
+
+PROP_GLUE_DATABASE = "Database"
+PROP_GLUE_DATABASE_LIST = "DatabaseList"
+PROP_GLUE_DATABASE_NAME = "Name"
+PROP_GLUE_DATABASE_LOCATION = "LocationUri"
+
+
+def _construct_parameters(metadata_location: str) -> Properties:
+    return {PROP_TABLE_TYPE: ICEBERG, PROP_METADATA_LOCATION: metadata_location}
+
+
+def _construct_table_input(table_name: str, metadata_location: str, properties: Properties) -> Dict[str, Any]:
+    table_input = {
+        PROP_GLUE_TABLE_NAME: table_name,
+        PROP_GLUE_TABLE_TYPE: EXTERNAL_TABLE_TYPE,
+        PROP_GLUE_TABLE_PARAMETERS: _construct_parameters(metadata_location),
+    }
+
+    if table_description := properties.get(PROP_GLUE_TABLE_DESCRIPTION):
+        table_input[PROP_GLUE_TABLE_DESCRIPTION] = table_description
+
+    return table_input
+
+
+def _convert_glue_to_iceberg(glue_table: Dict[str, Any], io: FileIO) -> Table:
+    properties: Properties = glue_table[PROP_GLUE_TABLE_PARAMETERS]
+
+    if PROP_TABLE_TYPE not in properties:
+        raise NoSuchTableError(
+            f"Property table_type missing, could not determine type: "
+            f"{glue_table[PROP_GLUE_TABLE_DATABASE_NAME]}.{glue_table[PROP_GLUE_TABLE_NAME]}"
+        )
+    glue_table_type = properties.get(PROP_TABLE_TYPE)
+    if glue_table_type != ICEBERG:

Review Comment:
   Thank you for your suggestion. I made everything in lowercase to make the check case insensitive.



-- 
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