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/11/07 17:58:42 UTC

[GitHub] [iceberg] Fokko commented on a diff in pull request #6131: Python: Add initial TableScan implementation

Fokko commented on code in PR #6131:
URL: https://github.com/apache/iceberg/pull/6131#discussion_r1015731779


##########
python/pyiceberg/table/__init__.py:
##########
@@ -14,30 +14,43 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-
-from typing import Dict, List, Optional
-
-from pydantic import Field
-
+from typing import (
+    ClassVar,
+    Dict,
+    List,
+    Optional,
+)
+
+from pyiceberg.expressions import AlwaysTrue, And, BooleanExpression
 from pyiceberg.schema import Schema
 from pyiceberg.table.metadata import TableMetadata
 from pyiceberg.table.partitioning import PartitionSpec
 from pyiceberg.table.snapshots import Snapshot, SnapshotLogEntry
 from pyiceberg.table.sorting import SortOrder
-from pyiceberg.typedef import Identifier
-from pyiceberg.utils.iceberg_base_model import IcebergBaseModel
+from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties
+
 
+class Table:
+    identifier: Identifier
+    metadata: TableMetadata
+    metadata_location: Optional[str]
 
-class Table(IcebergBaseModel):
-    identifier: Identifier = Field()
-    metadata_location: str = Field()
-    metadata: TableMetadata = Field()
+    def __init__(self, identifier: Identifier, metadata: TableMetadata, metadata_location: Optional[str]):
+        self.identifier = identifier
+        self.metadata = metadata
+        self.metadata_location = metadata_location
 
     def refresh(self):
         """Refresh the current table metadata"""
         raise NotImplementedError("To be implemented")
 
+    def name(self) -> Identifier:
+        """Return the identifer of this table"""
+        return self.identifier
+
+    def scan(self, **kwargs):

Review Comment:
   ```suggestion
       def scan(self, **kwargs) -> TableScan:
   ```



##########
python/pyiceberg/table/__init__.py:
##########
@@ -90,3 +103,90 @@ def snapshot_by_name(self, name: str) -> Optional[Snapshot]:
     def history(self) -> List[SnapshotLogEntry]:
         """Get the snapshot history of this table."""
         return self.metadata.snapshot_log
+
+
+class TableScan:
+    _always_true: ClassVar[BooleanExpression] = AlwaysTrue()
+    table: Table
+    row_filter: BooleanExpression
+    partition_filter: BooleanExpression
+    selected_fields: tuple[str]
+    case_sensitive: bool
+    snapshot_id: Optional[int]
+    options: Properties
+
+    def __init__(
+        self,
+        *,
+        table: Table,
+        row_filter: BooleanExpression = _always_true,
+        partition_filter: BooleanExpression = _always_true,
+        selected_fields: tuple[str] = ("*",),
+        case_sensitive: bool = True,
+        snapshot_id: Optional[int] = None,
+        options: Properties = EMPTY_DICT,
+    ):
+        self.table = table
+        self.row_filter = row_filter
+        self.partition_filter = partition_filter
+        self.selected_fields = selected_fields
+        self.case_sensitive = case_sensitive
+        self.snapshot_id = snapshot_id
+        self.options = options
+
+    def update(self, **overrides):
+        """Creates a copy of this table scan with updated fields."""
+        return TableScan(**{**self.__dict__, **overrides})
+
+    def snapshot(self):
+        if self.snapshot_id:
+            return self.table.snapshot_by_id(self.snapshot_id)
+
+        return self.table.current_snapshot()
+
+    def projection(self):

Review Comment:
   ```suggestion
       def projection(self) -> Schema:
   ```



##########
python/pyiceberg/table/__init__.py:
##########
@@ -14,30 +14,43 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-
-from typing import Dict, List, Optional
-
-from pydantic import Field
-
+from typing import (
+    ClassVar,
+    Dict,
+    List,
+    Optional,
+)
+
+from pyiceberg.expressions import AlwaysTrue, And, BooleanExpression
 from pyiceberg.schema import Schema
 from pyiceberg.table.metadata import TableMetadata
 from pyiceberg.table.partitioning import PartitionSpec
 from pyiceberg.table.snapshots import Snapshot, SnapshotLogEntry
 from pyiceberg.table.sorting import SortOrder
-from pyiceberg.typedef import Identifier
-from pyiceberg.utils.iceberg_base_model import IcebergBaseModel
+from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties
+
 
+class Table:

Review Comment:
   You could still override the method and it will work fine, only mypy will complain.



##########
python/pyiceberg/table/__init__.py:
##########
@@ -90,3 +103,90 @@ def snapshot_by_name(self, name: str) -> Optional[Snapshot]:
     def history(self) -> List[SnapshotLogEntry]:
         """Get the snapshot history of this table."""
         return self.metadata.snapshot_log
+
+
+class TableScan:
+    _always_true: ClassVar[BooleanExpression] = AlwaysTrue()

Review Comment:
   Not sure if this is necessary. `AlwaysTrue` is a singleton it will always return the same instance.



##########
python/pyiceberg/table/__init__.py:
##########
@@ -90,3 +103,90 @@ def snapshot_by_name(self, name: str) -> Optional[Snapshot]:
     def history(self) -> List[SnapshotLogEntry]:
         """Get the snapshot history of this table."""
         return self.metadata.snapshot_log
+
+
+class TableScan:
+    _always_true: ClassVar[BooleanExpression] = AlwaysTrue()
+    table: Table
+    row_filter: BooleanExpression
+    partition_filter: BooleanExpression
+    selected_fields: tuple[str]
+    case_sensitive: bool
+    snapshot_id: Optional[int]
+    options: Properties
+
+    def __init__(
+        self,
+        *,
+        table: Table,
+        row_filter: BooleanExpression = _always_true,
+        partition_filter: BooleanExpression = _always_true,
+        selected_fields: tuple[str] = ("*",),

Review Comment:
   ```suggestion
           selected_fields: Tuple[str] = ("*",),
   ```



##########
python/pyiceberg/table/__init__.py:
##########
@@ -90,3 +103,90 @@ def snapshot_by_name(self, name: str) -> Optional[Snapshot]:
     def history(self) -> List[SnapshotLogEntry]:
         """Get the snapshot history of this table."""
         return self.metadata.snapshot_log
+
+
+class TableScan:
+    _always_true: ClassVar[BooleanExpression] = AlwaysTrue()
+    table: Table
+    row_filter: BooleanExpression
+    partition_filter: BooleanExpression
+    selected_fields: tuple[str]
+    case_sensitive: bool
+    snapshot_id: Optional[int]
+    options: Properties
+
+    def __init__(
+        self,
+        *,
+        table: Table,
+        row_filter: BooleanExpression = _always_true,
+        partition_filter: BooleanExpression = _always_true,
+        selected_fields: tuple[str] = ("*",),
+        case_sensitive: bool = True,
+        snapshot_id: Optional[int] = None,
+        options: Properties = EMPTY_DICT,
+    ):
+        self.table = table
+        self.row_filter = row_filter
+        self.partition_filter = partition_filter
+        self.selected_fields = selected_fields
+        self.case_sensitive = case_sensitive
+        self.snapshot_id = snapshot_id
+        self.options = options
+
+    def update(self, **overrides):
+        """Creates a copy of this table scan with updated fields."""
+        return TableScan(**{**self.__dict__, **overrides})
+
+    def snapshot(self):

Review Comment:
   ```suggestion
       def snapshot(self) -> Snapshot:
   ```



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