Fokko commented on code in PR #453:
URL: https://github.com/apache/iceberg-python/pull/453#discussion_r1497670291


##########
pyiceberg/partitioning.py:
##########
@@ -193,6 +197,27 @@ def partition_type(self, schema: Schema) -> StructType:
             nested_fields.append(NestedField(field.field_id, field.name, 
result_type, required=False))
         return StructType(*nested_fields)
 
+    def partition_to_path(self, data: Record, schema: Schema) -> str:
+        partition_type = self.partition_type(schema)
+        field_types = partition_type.fields
+
+        pos = 0
+        field_strs = []
+        value_strs = []
+        for field_name in data._position_to_field_name:

Review Comment:
   Maybe use `enumerate` here to avoid having to increment the `pos`?



##########
tests/integration/test_partitioning_key.py:
##########
@@ -0,0 +1,722 @@
+# 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.
+# pylint:disable=redefined-outer-name
+from datetime import date, datetime
+from decimal import Decimal
+from typing import Any, List
+
+import pytest
+import pytz
+from pyspark.sql import SparkSession
+from pyspark.sql.utils import AnalysisException
+
+from pyiceberg.catalog import Catalog, load_catalog
+from pyiceberg.exceptions import NamespaceAlreadyExistsError
+from pyiceberg.partitioning import PartitionField, PartitionFieldValue, 
PartitionKey, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.transforms import (
+    BucketTransform,
+    DayTransform,
+    HourTransform,
+    IdentityTransform,
+    MonthTransform,
+    TruncateTransform,
+    YearTransform,
+)
+from pyiceberg.typedef import Record
+from pyiceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IntegerType,
+    LongType,
+    NestedField,
+    StringType,
+    TimestampType,
+    TimestamptzType,
+)
+
+
+@pytest.fixture()
+def catalog() -> Catalog:
+    catalog = load_catalog(
+        "local",
+        **{
+            "type": "rest",
+            "uri": "http://localhost:8181";,
+            "s3.endpoint": "http://localhost:9000";,
+            "s3.access-key-id": "admin",
+            "s3.secret-access-key": "password",
+        },
+    )
+
+    try:
+        catalog.create_namespace("default")
+    except NamespaceAlreadyExistsError:
+        pass
+
+    return catalog
+
+
+@pytest.fixture(scope="session")
+def session_catalog() -> Catalog:
+    return load_catalog(
+        "local",
+        **{
+            "type": "rest",
+            "uri": "http://localhost:8181";,
+            "s3.endpoint": "http://localhost:9000";,
+            "s3.access-key-id": "admin",
+            "s3.secret-access-key": "password",
+        },
+    )
+
+
+@pytest.fixture(scope="session")
+def spark() -> SparkSession:

Review Comment:
   We want to combine the configuration at some point.



##########
pyiceberg/partitioning.py:
##########
@@ -215,3 +240,53 @@ def assign_fresh_partition_spec_ids(spec: PartitionSpec, 
old_schema: Schema, fre
             )
         )
     return PartitionSpec(*partition_fields, spec_id=INITIAL_PARTITION_SPEC_ID)
+
+
+@dataclass(frozen=True)
+class PartitionFieldValue:
+    field: PartitionField
+    value: Any
+
+
+@dataclass(frozen=True)
+class PartitionKey:
+    raw_partition_field_values: List[PartitionFieldValue]
+    partition_spec: PartitionSpec
+    schema: Schema
+
+    @cached_property
+    def partition(self) -> Record:  # partition key in iceberg type
+        iceberg_typed_key_values = {}
+        for raw_partition_field_value in self.raw_partition_field_values:
+            partition_fields = 
self.partition_spec.source_id_to_fields_map[raw_partition_field_value.field.source_id]
+            assert len(partition_fields) == 1

Review Comment:
   We try to avoid asserts outside of the `tests/` directory.



##########
pyiceberg/partitioning.py:
##########
@@ -215,3 +240,53 @@ def assign_fresh_partition_spec_ids(spec: PartitionSpec, 
old_schema: Schema, fre
             )
         )
     return PartitionSpec(*partition_fields, spec_id=INITIAL_PARTITION_SPEC_ID)
+
+
+@dataclass(frozen=True)
+class PartitionFieldValue:
+    field: PartitionField
+    value: Any
+
+
+@dataclass(frozen=True)
+class PartitionKey:
+    raw_partition_field_values: List[PartitionFieldValue]
+    partition_spec: PartitionSpec
+    schema: Schema
+
+    @cached_property
+    def partition(self) -> Record:  # partition key in iceberg type
+        iceberg_typed_key_values = {}
+        for raw_partition_field_value in self.raw_partition_field_values:
+            partition_fields = 
self.partition_spec.source_id_to_fields_map[raw_partition_field_value.field.source_id]
+            assert len(partition_fields) == 1
+            partition_field = partition_fields[0]
+            iceberg_type = 
self.schema.find_field(name_or_id=raw_partition_field_value.field.source_id).field_type
+            iceberg_typed_value = _to_iceberg_type(iceberg_type, 
raw_partition_field_value.value)

Review Comment:
   I think we can re-use the literal logic here:
   ```suggestion
               iceberg_typed_value = 
literal(raw_partition_field_value.value).to(iceberg_type)
   ```



##########
pyiceberg/partitioning.py:
##########
@@ -193,6 +197,27 @@ def partition_type(self, schema: Schema) -> StructType:
             nested_fields.append(NestedField(field.field_id, field.name, 
result_type, required=False))
         return StructType(*nested_fields)
 
+    def partition_to_path(self, data: Record, schema: Schema) -> str:
+        partition_type = self.partition_type(schema)
+        field_types = partition_type.fields
+
+        pos = 0
+        field_strs = []
+        value_strs = []
+        for field_name in data._position_to_field_name:
+            value = getattr(data, field_name)

Review Comment:
   The concept behind a record is to have a positional-based lookup. This will 
yield he highest performance. The lookup based on the attr is mostly for tests.



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to