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/02/07 00:03:03 UTC

[GitHub] [iceberg] jun-he commented on a change in pull request #3450: [Python] support iceberg transforms in python

jun-he commented on a change in pull request #3450:
URL: https://github.com/apache/iceberg/pull/3450#discussion_r800252653



##########
File path: python/src/iceberg/transforms.py
##########
@@ -0,0 +1,583 @@
+# 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 re
+import struct
+from decimal import Decimal
+from enum import Enum
+from typing import Callable, Optional
+from uuid import UUID
+
+import mmh3  # type: ignore
+
+from iceberg.types import (
+    BinaryType,
+    DateType,
+    DecimalType,
+    FixedType,
+    IntegerType,
+    LongType,
+    StringType,
+    TimestampType,
+    TimestamptzType,
+    TimeType,
+    Type,
+    UUIDType,
+)
+from iceberg.utils import transform_util
+
+
+class Transform:
+    """Transform base class for concrete transforms.
+
+    A base class to transform values and project predicates on partition values.
+    This class is not used directly. Instead, use one of module method to create the child classes.
+
+    Args:
+        transform_string (str): name of the transform type
+        repr_string (str): string representation of a transform instance
+    """
+
+    def __init__(self, transform_string: str, repr_string: str):
+        self._transform_string = transform_string
+        self._repr_string = repr_string
+
+    def __repr__(self):
+        return self._repr_string
+
+    def __str__(self):
+        return self._transform_string
+
+    def __call__(self, value):
+        return self.apply(value)
+
+    def apply(self, value):
+        raise NotImplementedError()
+
+    def can_transform(self, target: Type) -> bool:
+        return False
+
+    def result_type(self, source: Type) -> Type:
+        return source
+
+    def preserves_order(self) -> bool:
+        return False
+
+    def satisfies_order_of(self, other) -> bool:
+        return self == other
+
+    def to_human_string(self, value) -> str:
+        if value is None:
+            return "null"
+        return str(value)
+
+    def dedup_name(self) -> str:
+        return self._transform_string
+
+
+class BaseBucketTransform(Transform):
+    """Base Transform class to transform a value into a bucket partition value
+
+    Transforms are parameterized by a number of buckets. Bucket partition transforms use a 32-bit
+    hash of the source value to produce a positive value by mod the bucket number.
+
+    Args:
+      source_type (Type): An Iceberg Type of IntegerType, LongType, DecimalType, DateType, TimeType,
+      TimestampType, TimestamptzType, StringType, BinaryType, FixedType, UUIDType.
+      num_buckets (int): The number of buckets.
+    """
+
+    def __init__(self, source_type: Type, num_buckets: int):
+        super().__init__(
+            f"bucket[{num_buckets}]",
+            f"transforms.bucket(source_type={repr(source_type)}, num_buckets={num_buckets})",
+        )
+        self._num_buckets = num_buckets
+
+    @property
+    def num_buckets(self) -> int:
+        return self._num_buckets
+
+    def hash(self, value) -> int:
+        raise NotImplementedError()
+
+    def apply(self, value) -> Optional[int]:
+        if value is None:
+            return None
+
+        return (self.hash(value) & IntegerType.max) % self._num_buckets
+
+    def can_transform(self, target: Type) -> bool:
+        raise NotImplementedError()
+
+    def result_type(self, source: Type) -> Type:
+        return IntegerType()
+
+
+class BucketIntegerTransform(BaseBucketTransform):
+    """Transforms a value of IntegerType or DateType into a bucket partition value
+
+    Example:
+        >>> transform = BucketIntegerTransform(100)
+        >>> transform.apply(34)
+        79
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return type(target) in {IntegerType, DateType}
+
+    def hash(self, value) -> int:
+        return mmh3.hash(struct.pack("q", value))
+
+
+class BucketLongTransform(BaseBucketTransform):
+    """Transforms a value of LongType, TimeType, TimestampType, or TimestamptzType
+    into a bucket partition value
+
+    Example:
+        >>> transform = BucketLongTransform(100)
+        >>> transform.apply(81068000000)
+        59
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return type(target) in {LongType, TimeType, TimestampType, TimestamptzType}
+
+    def hash(self, value) -> int:
+        return mmh3.hash(struct.pack("q", value))
+
+
+class BucketDoubleTransform(BaseBucketTransform):
+    """Transforms a value of FloatType or DoubleType into a bucket partition value.
+
+    Note that bucketing by Double is not allowed by the spec, but this has the hash implementation.
+
+    Example:
+        >>> transform = BucketDoubleTransform(8)
+        >>> transform.hash(1.0)
+        -142385009
+    """
+
+    def hash(self, value) -> int:
+        return mmh3.hash(struct.pack("d", value))
+
+
+class BucketDecimalTransform(BaseBucketTransform):
+    """Transforms a value of DecimalType into a bucket partition value.
+
+    Example:
+        >>> transform = BucketDecimalTransform(100)
+        >>> transform.apply(Decimal("14.20"))
+        59
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return isinstance(target, DecimalType)
+
+    def hash(self, value: Decimal) -> int:
+        return mmh3.hash(transform_util.decimal_to_bytes(value))
+
+
+class BucketStringTransform(BaseBucketTransform):
+    """Transforms a value of StringType into a bucket partition value.
+
+    Example:
+        >>> transform = BucketStringTransform(100)
+        >>> transform.apply("iceberg")
+        89
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return isinstance(target, StringType)
+
+    def hash(self, value: str) -> int:
+        return mmh3.hash(value)
+
+
+class BucketFixedTransform(BaseBucketTransform):
+    """Transforms a value of FixedType into a bucket partition value.
+
+    Example:
+        >>> transform = BucketFixedTransform(128)
+        >>> transform.apply(b"foo")
+        32
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return isinstance(target, FixedType)
+
+    def hash(self, value: bytearray) -> int:
+        return mmh3.hash(value)
+
+
+class BucketBinaryTransform(BaseBucketTransform):
+    """Transforms a value of BinaryType into a bucket partition value.
+
+    Example:
+        >>> transform = BucketBinaryTransform(128)
+        >>> transform.apply(b"\x00\x01\x02\x03")
+        57
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return isinstance(target, BinaryType)
+
+    def hash(self, value: bytes) -> int:
+        return mmh3.hash(value)
+
+
+class BucketUUIDTransform(BaseBucketTransform):
+    """Transforms a value of UUIDType into a bucket partition value.
+
+    Example:
+        >>> transform = BucketUUIDTransform(100)
+        >>> transform.apply(UUID("f79c3e09-677c-4bbd-a479-3f349cb785e7"))
+        40
+    """
+
+    def can_transform(self, target: Type) -> bool:
+        return isinstance(target, UUIDType)
+
+    def hash(self, value: UUID) -> int:
+        return mmh3.hash(
+            struct.pack(
+                ">QQ",
+                (value.int >> 64) & 0xFFFFFFFFFFFFFFFF,
+                value.int & 0xFFFFFFFFFFFFFFFF,
+            )
+        )
+
+
+class DateTimeTransform(Transform):
+    """Base transform class for transforms of DateType, TimestampType, and TimestamptzType."""
+
+    class Granularity(Enum):
+        def __init__(
+            self, order: int, result_type: Type, human_string: Callable[[int], str]
+        ):
+            self.order = order
+            self.result_type = result_type
+            self.human_string = human_string
+
+        YEAR = 3, IntegerType(), transform_util.human_year
+        MONTH = 2, IntegerType(), transform_util.human_month
+        DAY = 1, DateType(), transform_util.human_day
+        HOUR = 0, IntegerType(), transform_util.human_hour
+
+    _DATE_APPLY_FUNCS = {
+        Granularity.YEAR: transform_util.years_for_days,
+        Granularity.MONTH: transform_util.months_for_days,
+        Granularity.DAY: lambda d: d,
+    }
+
+    _TIMESTAMP_APPLY_FUNCS = {
+        Granularity.YEAR: transform_util.years_for_ts,
+        Granularity.MONTH: transform_util.months_for_ts,
+        Granularity.DAY: transform_util.days_for_ts,
+        Granularity.HOUR: transform_util.hours_for_ts,
+    }
+
+    def __init__(self, source_type: Type, name: str):
+        super().__init__(name, f"transforms.{name}(source_type={repr(source_type)})")
+
+        self._type = source_type
+        try:
+            self._granularity = DateTimeTransform.Granularity[name.upper()]
+
+            if isinstance(source_type, DateType):
+                self._apply = DateTimeTransform._DATE_APPLY_FUNCS[self._granularity]
+            elif type(source_type) in {TimestampType, TimestamptzType}:
+                self._apply = DateTimeTransform._TIMESTAMP_APPLY_FUNCS[
+                    self._granularity
+                ]
+            else:
+                raise KeyError
+        except KeyError:
+            raise ValueError(f"Cannot partition type {source_type} by {name}")
+
+    def __eq__(self, other):
+        if type(self) is type(other):
+            return self._type == other._type and self._granularity == other._granularity
+        return False
+
+    def can_transform(self, target: Type) -> bool:
+        if isinstance(self._type, DateType):
+            return isinstance(target, DateType)
+        else:  # self._type is either TimestampType or TimestamptzType
+            return not isinstance(target, DateType)
+
+    def apply(self, value: int) -> int:
+        return self._apply(value)
+
+    def result_type(self, source_type: Type) -> Type:
+        return self._granularity.result_type
+
+    def preserves_order(self) -> bool:
+        return True
+
+    def satisfies_order_of(self, other: Transform) -> bool:
+        if self == other:
+            return True
+
+        if isinstance(other, DateTimeTransform):
+            return self._granularity.order <= other._granularity.order
+
+        return False
+
+    def to_human_string(self, value) -> str:
+        if value is None:
+            return "null"
+        return self._granularity.human_string(value)
+
+    def dedup_name(self) -> str:
+        return "time"
+
+
+class IdentityTransform(Transform):
+    def __init__(self, source_type: Type):
+        super().__init__(
+            "identity",
+            f"transforms.identity(source_type={repr(source_type)})",
+        )
+        self._type = source_type
+
+    def apply(self, value):
+        return value
+
+    def can_transform(self, target: Type) -> bool:
+        return target.is_primitive
+
+    def preserves_order(self) -> bool:
+        return True
+
+    def satisfies_order_of(self, other: Transform) -> bool:
+        return other.preserves_order()
+
+    def to_human_string(self, value) -> str:
+        if value is None:
+            return "null"
+        elif isinstance(self._type, DateType):
+            return transform_util.human_day(value)
+        elif isinstance(self._type, TimeType):
+            return transform_util.human_time(value)
+        elif isinstance(self._type, TimestampType):
+            return transform_util.human_timestamp(value)
+        elif isinstance(self._type, TimestamptzType):
+            return transform_util.human_timestamptz(value)
+        elif isinstance(self._type, FixedType):
+            return transform_util.base64encode(value)
+        elif isinstance(self._type, BinaryType):
+            return transform_util.base64encode(value)
+        else:
+            return str(value)
+
+
+class TruncateTransform(Transform):
+    """A transform for truncating a value to a specified width.
+
+    Args:
+      source_type (Type): An Iceberg Type of IntegerType, LongType, StringType, or BinaryType
+      width (int): The truncate width
+
+    Raises:
+      ValueError: If a type is provided that is incompatible with a Truncate transform
+    """
+
+    _VALID_TYPES = {IntegerType, LongType, StringType, BinaryType, DecimalType}
+
+    def __init__(self, source_type: Type, width: int):
+        if type(source_type) not in TruncateTransform._VALID_TYPES:
+            raise ValueError(f"Cannot truncate type: {source_type}")
+
+        super().__init__(
+            f"truncate[{width}]",
+            f"transforms.truncate(source_type={repr(source_type)}, width={width})",
+        )
+        self._type = source_type
+        self._width = width
+
+        if type(self._type) in {IntegerType, LongType}:
+            self._apply = lambda v, w: v - v % w
+        elif type(self._type) in {StringType, BinaryType}:
+            self._apply = lambda v, w: v[0 : min(w, len(v))]

Review comment:
       SG, moved both into transform_util. Also, the other option is to add truncate method to types, then we don't need `if ... elif ... else` 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