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/12/15 17:06:02 UTC

[GitHub] [iceberg] Fokko opened a new pull request, #6437: Python: Projection by Field ID

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

   instead of 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] Fokko commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done

Review Comment:
   Good one, also updated the one in `schema.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] Fokko commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:
+            return values.cast(schema_to_pyarrow(promote(file_field.field_type, field.field_type)))
+
+        return values
+
+    def schema(self, schema: Schema, schema_partner: Optional[pa.Array], struct_result: Optional[pa.Array]) -> Optional[pa.Array]:
+        return struct_result
+
+    def struct(
+        self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
+    ) -> Optional[pa.Array]:
+        if struct_array is None:
+            return None
+        return pa.StructArray.from_arrays(arrays=field_results, fields=pa.struct(schema_to_pyarrow(struct)))
+
+    def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
+        if field_array is not None:
+            return self.cast_if_needed(field, field_array)
+        elif field.optional:
+            arrow_type = schema_to_pyarrow(field.field_type)
+            return pa.nulls(self.table_length, type=arrow_type)

Review Comment:
   This is great, thanks for the thorough explanation. It makes sense now. I've also copied the test case into the source.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")

Review Comment:
   Missing Iceberg schema for file: {path}? That avoids negation and needing to understand why it would be "encoded" in a 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -44,8 +49,14 @@
     S3FileSystem,
 )
 
-from pyiceberg.expressions import BooleanExpression, BoundTerm, Literal
-from pyiceberg.expressions.visitors import BoundBooleanExpressionVisitor
+from pyiceberg.avro.resolver import promote

Review Comment:
   Looks like this method should be moved since it isn't specific to Avro.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:

Review Comment:
   Nevermind, the next test covers this.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:

Review Comment:
   Would it make sense to also project the original int column for this table and verify it has the expected values?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:
+        """Visit a Schema with a concrete SchemaVisitorWithPartner"""
+        struct_result = self.visit_with_arrow(requested_schema.as_struct(), file_schema.as_struct())
+        pyarrow_schema = schema_to_pyarrow(requested_schema)
+        return pa.Table.from_arrays(struct_result.flatten(), schema=pyarrow_schema)

Review Comment:
   What is this doing? Why does it visit and then allocate something?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:
+        """Visit a Schema with a concrete SchemaVisitorWithPartner"""
+        struct_result = self.visit_with_arrow(requested_schema.as_struct(), file_schema.as_struct())
+        pyarrow_schema = schema_to_pyarrow(requested_schema)
+        return pa.Table.from_arrays(struct_result.flatten(), schema=pyarrow_schema)
+
+    def _get_field_by_id(self, field_id: int) -> Optional[NestedField]:
+        try:
+            return self.file_schema.find_field(field_id)
+        except ValueError:
+            # Field is not in the file
+            return None
+
+    @visit_with_arrow.register(StructType)
+    def _(self, requested_struct: StructType, file_struct: Optional[IcebergType]) -> pa.Array:  # pylint: disable=unused-argument
+        """Visit a StructType with a concrete SchemaVisitorWithPartner"""
+        results = []
+
+        for requested_field in requested_struct.fields:
+            file_field = self._get_field_by_id(requested_field.field_id)
+
+            if file_field is None and requested_field.required:
+                raise ResolveException(f"Field is required, and could not be found in the file: {requested_field}")
+
+            results.append(self.visit_with_arrow(requested_field.field_type, file_field))
+
+        pyarrow_schema = schema_to_pyarrow(requested_struct)
+        return pa.StructArray.from_arrays(arrays=results, fields=pyarrow_schema)
+
+    @visit_with_arrow.register(ListType)
+    def _(self, requested_list: ListType, file_field: Optional[NestedField]) -> pa.Array:
+        """Visit a ListType with a concrete SchemaVisitorWithPartner"""
+
+        if file_field is not None:
+            if not isinstance(file_field.field_type, ListType):
+                raise ValueError(f"Expected list, got: {file_field}")
+
+            return self.visit_with_arrow(requested_list.element_type, self._get_field_by_id(file_field.field_type.element_id))
+        else:
+            # Not in the file, fill in with nulls
+            return pa.nulls(len(self.table), type=pa.list_(schema_to_pyarrow(requested_list.element_type)))
+
+    @visit_with_arrow.register(MapType)
+    def _(self, requested_map: MapType, file_map: Optional[NestedField]) -> pa.Array:
+        """Visit a MapType with a concrete SchemaVisitorWithPartner"""
+
+        if file_map is not None:
+            if not isinstance(file_map.field_type, MapType):
+                raise ValueError(f"Expected map, got: {file_map}")
+
+            key = self._get_field_by_id(file_map.field_type.key_id)
+            return self.visit_with_arrow(requested_map.key_type, key)
+        else:
+            # Not in the file, fill in with nulls
+            return pa.nulls(
+                len(self.table),
+                type=pa.map_(schema_to_pyarrow(requested_map.key_type), schema_to_pyarrow(requested_map.value_type)),
+            )
+
+    def _get_column_data(self, file_field: NestedField) -> pa.Array:
+        column_name = self.file_schema.find_column_name(file_field.field_id)
+        column_data = self.table
+        struct_schema = self.table.schema
+
+        if column_name is None:
+            # Should not happen
+            raise ValueError(f"Could not find column: {column_name}")
+
+        column_parts = list(reversed(column_name.split(".")))
+        while len(column_parts) > 1:
+            part = column_parts.pop()
+            column_data = column_data.column(part)
+            struct_schema = struct_schema[struct_schema.get_field_index(part)].type
+
+        if not isinstance(struct_schema, (pa.ListType, pa.MapType)):
+            # PyArrow does not have an element
+            idx = struct_schema.get_field_index(column_parts.pop())
+            column_data = column_data.flatten()[idx]
+
+        return column_data.combine_chunks()

Review Comment:
   I had to use `combine_chunks()` in my PR as well when reading columns from the table. I'm a bit concerned about that because it may cause copies and reallocation. I'm not sure a way to avoid it though.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:
+            return values.cast(schema_to_pyarrow(promote(file_field.field_type, field.field_type)))
+
+        return values
+
+    def schema(self, schema: Schema, schema_partner: Optional[pa.Array], struct_result: Optional[pa.Array]) -> Optional[pa.Array]:
+        return struct_result
+
+    def struct(
+        self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
+    ) -> Optional[pa.Array]:
+        if struct_array is None:
+            return None
+        return pa.StructArray.from_arrays(arrays=field_results, fields=pa.struct(schema_to_pyarrow(struct)))
+
+    def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
+        if field_array is not None:
+            return self.cast_if_needed(field, field_array)
+        elif field.optional:
+            arrow_type = schema_to_pyarrow(field.field_type)
+            return pa.nulls(self.table_length, type=arrow_type)

Review Comment:
   This isn't correct. The array length may not necessarily be the number of rows if the array is in a list element, map key, or map value. Here's a test case that demonstrates the problem:
   
   ```python
   @pytest.fixture
   def schema_list_of_structs() -> Schema:
       return Schema(
           NestedField(5, "locations", ListType(51, StructType(
               NestedField(511, "lat", DoubleType()),
               NestedField(512, "long", DoubleType())
           ), element_required=False), required=False),
       )
   
   
   @pytest.fixture
   def file_list_of_structs(schema_list_of_structs: Schema, tmpdir: str) -> str:
       pyarrow_schema = pa.schema(
           schema_to_pyarrow(schema_list_of_structs),
           metadata={"iceberg.schema": schema_list_of_structs.json()})
       return _write_table_to_file(
           f"file:{tmpdir}/e.parquet",
           pyarrow_schema,
           pa.Table.from_pylist(
               [
                   {"locations": [{"lat": 52.371807, "long": 4.896029}, {"lat": 52.387386, "long": 4.646219}]},
                   {"locations": []},
                   {"locations": [{"lat": 52.078663, "long": 4.288788}, {"lat": 52.387386, "long": 4.646219}]},
               ],
               schema=pyarrow_schema,
           ),
       )
   
   
   def test_projection_list_of_structs(schema_list_of_structs: Schema, file_list_of_structs: str) -> None:
       schema = Schema(
           NestedField(5, "locations", ListType(51, StructType(
               NestedField(511, "latitude", DoubleType()),
               NestedField(512, "longitude", DoubleType()),
               NestedField(513, "altitude", DoubleType(), required=False)
           ), element_required=False), required=False),
       )
   
       result_table = project(schema, [file_list_of_structs])
       assert len(result_table.columns) == 1
       assert len(result_table.columns[0]) == 3
       for actual, expected in zip(result_table.columns[0], [
           [{"latitude": 52.371807, "longitude": 4.896029, "altitude": None}, {"latitude": 52.387386, "longitude": 4.646219, "altitude": None}],
           [],
           [{"latitude": 52.078663, "longitude": 4.288788, "altitude": None}, {"latitude": 52.387386, "longitude": 4.646219, "altitude": None}],
       ]):
           assert actual.as_py() == expected
       assert (
           repr(result_table.schema) == """locations: list<item: struct<latitude: double not null, longitude: double not null, altitude: double>>
     child 0, item: struct<latitude: double not null, longitude: double not null, altitude: double>
         child 0, latitude: double not null
         child 1, longitude: double not null
         child 2, altitude: double"""
       )
   ```
   
   If this uses the table length, then the test fails with `pyarrow.lib.ArrowInvalid: Mismatching child array lengths`.
   
   This works when you use `len(struct_array)`, which is why I originally had this logic inside the `struct` method:
   
   ```python
       def struct(
           self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
       ) -> Optional[pa.Array]:
           if struct_array is None:
               return None
           field_arrays: List[pa.Array] = []
           fields: List[pa.Field] = []
           for field, field_array in zip(struct.fields, field_results):
               if field_array is not None:
                   array = self.cast_if_needed(field, field_array)
                   field_arrays.append(array)
                   fields.append(pa.field(field.name, array.type, field.optional))
               elif field.optional:
                   arrow_type = schema_to_pyarrow(field.field_type)
                   field_arrays.append(pa.nulls(len(struct_array), type=arrow_type))
                   fields.append(pa.field(field.name, arrow_type, field.optional))
               else:
                   raise ResolveError(f"Field is required, and could not be found in the file: {field}")
   
           return pa.StructArray.from_arrays(arrays=field_arrays, fields=pa.struct(fields))
   
       def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
           return field_array
   ```
   
   I also think that it is better to put this logic in the `struct` method to avoid calling `schema_to_pyarrow(struct)`. In the version above, the struct is built from each field rather than converting another tree-like structure in the middle of visiting.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:

Review Comment:
   Ah I see, this also allows us to remove the struct from the promotion. Thanks!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(

Review Comment:
   Has been added 👍🏻 



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")
+
+
+@promote.register(IntegerType)
+def _(file_type: IntegerType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, LongType):
+        # Ints/Longs are binary compatible in Avro, so this is okay
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an int to {read_type}")
+
+
+@promote.register(FloatType)
+def _(file_type: FloatType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DoubleType):
+        # A double type is wider
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an float to {read_type}")
+
+
+@promote.register(StringType)
+def _(file_type: StringType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, BinaryType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an string to {read_type}")
+
+
+@promote.register(BinaryType)
+def _(file_type: BinaryType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, StringType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an binary to {read_type}")
+
+
+@promote.register(DecimalType)
+def _(file_type: DecimalType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DecimalType):
+        if file_type.precision <= read_type.precision and file_type.scale == file_type.scale:
+            return read_type
+        else:
+            raise ResolveException(f"Cannot reduce precision from {file_type} to {read_type}")
+    else:
+        raise ResolveException(f"Cannot promote an decimal to {read_type}")
+
+
+@promote.register(StructType)
+def _(file_type: StructType, read_type: IcebergType) -> IcebergType:

Review Comment:
   Do we need a struct implementation? Seems like this should only be for primitives to keep the logic simple. Visitors can line up other types.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None

Review Comment:
   Good one, added!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[
+                    Schema(
+                        # Reuses the id 1
+                        NestedField(1, "other_id", IntegerType())
+                    )
+                ],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=0,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("other_id", 1),
+        Schema(
+            # Reuses the id 1
+            NestedField(1, "other_id", IntegerType())
+        ),
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 1  # Just 2
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_filter_add_column(schema_int: Schema, table_int: str, table_str: str) -> None:
+    """We have one file that has the column, and the other one doesn't"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+            FileScanTask(
+                DataFile(file_path=table_str, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ],

Review Comment:
   I like this set of files.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0

Review Comment:
   Why are there no columns? The buffer should still have the same structure, it should just be empty 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)

Review Comment:
   I like this a lot!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")

Review Comment:
   Fair point, I've changed this to: `Missing Iceberg schema in Metadata for file: {path}`. I think it is important to also mention that it is not found in the 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):

Review Comment:
   Can we support other FileIO implementations? I think that we want to eventually and we know that Arrow can read streams that are created by others.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)

Review Comment:
   Nice!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())

Review Comment:
   The original name is `id`:
   ```python
   @pytest.fixture
   def schema_int() -> Schema:
       return Schema(NestedField(1, "id", IntegerType(), required=False))
   ```
   I've updated the testcase



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:

Review Comment:
   I've created a special visitor for this one.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:

Review Comment:
   I think the inner part of this loop should be a Parquet method that we provide, so that the caller can read progressively or read parts in parallel tasks. This is a great start for single process, though.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +757,68 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ExpressionProjector(BooleanExpressionVisitor[BooleanExpression]):
+    """Rewrites a boolean expression by replacing unbound references with references to fields in a struct schema
+
+    Args:
+      table_schema (Schema): The schema of the table
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case a predicate is already bound
+    """
+
+    table_schema: Schema
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, table_schema: Schema, file_schema: Schema, case_sensitive: bool) -> None:
+        self.table_schema = table_schema
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        if not isinstance(predicate.term, Reference):
+            raise ValueError(f"Exprected reference: {predicate.term}")
+
+        field = self.table_schema.find_field(predicate.term.name, case_sensitive=self.case_sensitive)
+        file_column_name = self.file_schema.find_column_name(field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")
+
+        if isinstance(predicate, UnaryPredicate):
+            return predicate.__class__(Reference(file_column_name))

Review Comment:
   Oh, nevermind. I see that the input expression is unbound. I'm not sure whether it would be better to do this to a bound expression or not. That seems like good separation of responsibilities and a good way to have consistent error messages when an expression isn't valid for a particular schema.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")

Review Comment:
   What about when this is called with the same type? How about this:
   
   ```python
       if file_type == read_type:
           return file_type
       else:
           raise ResolveError(f"Cannot promote {file_type} to {read_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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:
+            return values.cast(schema_to_pyarrow(promote(file_field.field_type, field.field_type)))
+
+        return values
+
+    def schema(self, schema: Schema, schema_partner: Optional[pa.Array], struct_result: Optional[pa.Array]) -> Optional[pa.Array]:
+        return struct_result
+
+    def struct(
+        self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
+    ) -> Optional[pa.Array]:
+        if struct_array is None:
+            return None
+        return pa.StructArray.from_arrays(arrays=field_results, fields=pa.struct(schema_to_pyarrow(struct)))
+
+    def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
+        if field_array is not None:
+            return self.cast_if_needed(field, field_array)
+        elif field.optional:
+            arrow_type = schema_to_pyarrow(field.field_type)
+            return pa.nulls(self.table_length, type=arrow_type)

Review Comment:
   Since I already have this working, I opened another PR: https://github.com/Fokko/incubator-iceberg/pull/342



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)

Review Comment:
   Good suggestion, the current error is really obscure. I've added a check and created a ticket https://github.com/apache/iceberg/issues/6505



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:

Review Comment:
   This is a part of my latest PR.



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

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

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


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


[GitHub] [iceberg] Fokko commented on pull request #6437: Python: Projection by Field ID

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

   Thanks for the review @rdblue 


-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):

Review Comment:
   Should we keep that for another PR?



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:

Review Comment:
   I like requested a lot, thanks!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyproject.toml:
##########
@@ -222,5 +222,9 @@ ignore_missing_imports = true
 module = "pyparsing.*"
 ignore_missing_imports = true
 
+[[tool.mypy.overrides]]
+module = "numpy.*"
+ignore_missing_imports = true

Review Comment:
   PyArrow ships with numpy. We use it to create empty arrays, but I think we can remove that!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())

Review Comment:
   What about `other_name` instead of `other_id`? The ID matches.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null

Review Comment:
   The location struct should be optional because it isn't present right?
   
   If a required field is not present in a data file, then that should be an error.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[
+                    Schema(

Review Comment:
   Yes, I've cleaned this up



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")
+
+
+@promote.register(IntegerType)
+def _(file_type: IntegerType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, LongType):

Review Comment:
   Do these also need to handle the case where file_type and read_type are equal?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/exceptions.py:
##########
@@ -86,3 +86,7 @@ class NotInstalledError(Exception):
 
 class SignError(Exception):
     """Raises when unable to sign a S3 request"""
+
+
+class ResolveException(Exception):

Review Comment:
   `ResolveError`?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:
+            return values.cast(schema_to_pyarrow(promote(file_field.field_type, field.field_type)))
+
+        return values
+
+    def schema(self, schema: Schema, schema_partner: Optional[pa.Array], struct_result: Optional[pa.Array]) -> Optional[pa.Array]:
+        return struct_result
+
+    def struct(
+        self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
+    ) -> Optional[pa.Array]:
+        if struct_array is None:
+            return None
+        return pa.StructArray.from_arrays(arrays=field_results, fields=pa.struct(schema_to_pyarrow(struct)))
+
+    def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
+        if field_array is not None:
+            return self.cast_if_needed(field, field_array)
+        elif field.optional:
+            arrow_type = schema_to_pyarrow(field.field_type)
+            return pa.nulls(self.table_length, type=arrow_type)

Review Comment:
   This isn't correct. The array length may not necessarily be the number of rows if the array is in a list element, map key, or map value. Here's a test case that demonstrates the problem:
   
   ```python
   @pytest.fixture
   def schema_list_of_structs() -> Schema:
       return Schema(
           NestedField(5, "locations", ListType(51, StructType(
               NestedField(511, "lat", DoubleType()),
               NestedField(512, "long", DoubleType())
           ), element_required=False), required=False),
       )
   
   
   @pytest.fixture
   def file_list_of_structs(schema_list_of_structs: Schema, tmpdir: str) -> str:
       pyarrow_schema = pa.schema(
           schema_to_pyarrow(schema_list_of_structs),
           metadata={"iceberg.schema": schema_list_of_structs.json()})
       return _write_table_to_file(
           f"file:{tmpdir}/e.parquet",
           pyarrow_schema,
           pa.Table.from_pylist(
               [
                   {"locations": [{"lat": 52.371807, "long": 4.896029}, {"lat": 52.387386, "long": 4.646219}]},
                   {"locations": []},
                   {"locations": [{"lat": 52.078663, "long": 4.288788}, {"lat": 52.387386, "long": 4.646219}]},
               ],
               schema=pyarrow_schema,
           ),
       )
   
   
   def test_projection_list_of_structs(schema_list_of_structs: Schema, file_list_of_structs: str) -> None:
       schema = Schema(
           NestedField(5, "locations", ListType(51, StructType(
               NestedField(511, "latitude", DoubleType()),
               NestedField(512, "longitude", DoubleType()),
               NestedField(513, "altitude", DoubleType(), required=False)
           ), element_required=False), required=False),
       )
   
       result_table = project(schema, [file_list_of_structs])
       assert len(result_table.columns) == 1
       assert len(result_table.columns[0]) == 3
       for actual, expected in zip(result_table.columns[0], [
           [{"latitude": 52.371807, "longitude": 4.896029, "altitude": None}, {"latitude": 52.387386, "longitude": 4.646219, "altitude": None}],
           [],
           [{"latitude": 52.078663, "longitude": 4.288788, "altitude": None}, {"latitude": 52.387386, "longitude": 4.646219, "altitude": None}],
       ]):
           assert actual.as_py() == expected
       assert (
           repr(result_table.schema) == """locations: list<item: struct<latitude: double not null, longitude: double not null, altitude: double>>
     child 0, item: struct<latitude: double not null, longitude: double not null, altitude: double>
         child 0, latitude: double not null
         child 1, longitude: double not null
         child 2, altitude: double"""
       )
   ```
   
   If this uses the table length, then the test fails with `pyarrow.lib.ArrowInvalid: Mismatching child array lengths`.
   
   This works when you use `len(struct_array)`, which is why I originally had this logic inside the `struct` method:
   
   ```python
       def struct(
           self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
       ) -> Optional[pa.Array]:
           if struct_array is None:
               return None
           field_arrays: List[pa.Array] = []
           fields: List[pa.Field] = []
           for field, field_array in zip(struct.fields, field_results):
               if field_array is not None:
                   array = self.cast_if_needed(field, field_array)
                   field_arrays.append(array)
                   fields.append(pa.field(field.name, array.type, field.optional))
               elif field.optional:
                   arrow_type = schema_to_pyarrow(field.field_type)
                   field_arrays.append(pa.nulls(len(struct_array), type=arrow_type))
                   fields.append(pa.field(field.name, arrow_type, field.optional))
               else:
                   raise ResolveError(f"Field is required, and could not be found in the file: {field}")
   
           return pa.StructArray.from_arrays(arrays=field_arrays, fields=pa.struct(fields))
   ```
   
   I also think that it is better to put this logic in the `struct` method to avoid calling `schema_to_pyarrow(struct)`. In the version above, the struct is built from each field rather than converting another tree-like structure in the middle of visiting.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:

Review Comment:
   This needs to be updated to check whether the expected type is a primitive. Otherwise, when you have nested structs or other complex types, it fails because `promote` doesn't support them.
   
   ```python
       def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
           file_field = self.file_schema.find_field(field.field_id)
           if field.field_type.is_primitive and field.field_type != file_field.field_type:
               return values.cast(schema_to_pyarrow(promote(file_field.field_type, field.field_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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:

Review Comment:
   This is allowed to return a different type than the rest of the methods?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,

Review Comment:
   Can you set these as `schema_int.schema_id`? That way we don't break tests if something changes.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:

Review Comment:
   Should this be `file_int`? It doesn't seem like a table to me.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)

Review Comment:
   Good catch, this wasn't working, just fixed this 👍🏻 



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",

Review Comment:
   That's correct!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,61 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      table_schema (Schema): The schema of the table

Review Comment:
   I've been looking for a checker for a while, but haven't found one yet 😭 



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:

Review Comment:
   This needs to check whether `file_schema` is a `Schema` 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] Fokko commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")
+
+
+@promote.register(IntegerType)
+def _(file_type: IntegerType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, LongType):
+        # Ints/Longs are binary compatible in Avro, so this is okay
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an int to {read_type}")
+
+
+@promote.register(FloatType)
+def _(file_type: FloatType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DoubleType):
+        # A double type is wider
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an float to {read_type}")
+
+
+@promote.register(StringType)
+def _(file_type: StringType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, BinaryType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an string to {read_type}")
+
+
+@promote.register(BinaryType)
+def _(file_type: BinaryType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, StringType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an binary to {read_type}")
+
+
+@promote.register(DecimalType)
+def _(file_type: DecimalType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DecimalType):
+        if file_type.precision <= read_type.precision and file_type.scale == file_type.scale:
+            return read_type
+        else:
+            raise ResolveException(f"Cannot reduce precision from {file_type} to {read_type}")
+    else:
+        raise ResolveException(f"Cannot promote an decimal to {read_type}")
+
+
+@promote.register(StructType)
+def _(file_type: StructType, read_type: IcebergType) -> IcebergType:

Review Comment:
   I would say we can rewrite:
   ```python
       def struct(
           self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
       ) -> Optional[pa.Array]:
           if struct_array is None:
               return None
   
           field_arrays: List[pa.Array] = []
           fields: List[pa.Field] = []
           for pos, field_array in enumerate(field_results):
               field = struct.fields[pos]
               if field_array is not None:
                   array = self.cast_if_needed(field, field_array)
                   field_arrays.append(array)
                   fields.append(pa.field(field.name, array.type, field.optional))
               elif field.optional:
                   arrow_type = schema_to_pyarrow(field.field_type)
                   field_arrays.append(pa.nulls(len(struct_array)).cast(arrow_type))
                   fields.append(pa.field(field.name, arrow_type, field.optional))
               else:
                   raise ResolveException(f"Field is required, and could not be found in the file: {field}")
   
           return pa.StructArray.from_arrays(arrays=field_arrays, fields=pa.struct(fields))
   
       def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
           return field_array
   ```
   I think this is cleaner because we just handle the field on the field method:
   
   ```python
       def struct(
           self, struct: StructType, struct_array: Optional[pa.Array], field_results: List[Optional[pa.Array]]
       ) -> Optional[pa.Array]:
           if struct_array is None:
               return None
           return pa.StructArray.from_arrays(arrays=field_results, fields=pa.struct(schema_to_pyarrow(struct)))
   
       def field(self, field: NestedField, _: Optional[pa.Array], field_array: Optional[pa.Array]) -> Optional[pa.Array]:
           if field_array is not None:
               return self.cast_if_needed(field, field_array)
           elif field.optional:
               arrow_type = schema_to_pyarrow(field.field_type)
               return pa.nulls(3, type=arrow_type)  # We need to find the length somehow
           else:
               raise ResolveError(f"Field is required, and could not be found in the file: {field}")
   ```
   
   But then the field_type can still be a StructType



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))

Review Comment:
   Okay, so this is because of the use of `select_full_types=False`.
   
   I think that this works since it is using `schema.field_ids` but we may want to move this to a helper method eventually, rather than embedding it 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")
+
+
+@promote.register(IntegerType)
+def _(file_type: IntegerType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, LongType):
+        # Ints/Longs are binary compatible in Avro, so this is okay
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an int to {read_type}")
+
+
+@promote.register(FloatType)
+def _(file_type: FloatType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DoubleType):
+        # A double type is wider
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an float to {read_type}")
+
+
+@promote.register(StringType)
+def _(file_type: StringType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, BinaryType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an string to {read_type}")
+
+
+@promote.register(BinaryType)
+def _(file_type: BinaryType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, StringType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an binary to {read_type}")
+
+
+@promote.register(DecimalType)
+def _(file_type: DecimalType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DecimalType):
+        if file_type.precision <= read_type.precision and file_type.scale == file_type.scale:
+            return read_type
+        else:
+            raise ResolveException(f"Cannot reduce precision from {file_type} to {read_type}")
+    else:
+        raise ResolveException(f"Cannot promote an decimal to {read_type}")
+
+
+@promote.register(StructType)
+def _(file_type: StructType, read_type: IcebergType) -> IcebergType:

Review Comment:
   I don't think that we want to do this. For nested structures, visitors should handle the logic. Type promotion should just be for primitive types. I handled this in the code above by checking whether the type was primitive before calling. I think that's better than trying to add logic to implement promotion, which is not defined for structs.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:
+        """Visit a Schema with a concrete SchemaVisitorWithPartner"""
+        struct_result = self.visit_with_arrow(requested_schema.as_struct(), file_schema.as_struct())
+        pyarrow_schema = schema_to_pyarrow(requested_schema)
+        return pa.Table.from_arrays(struct_result.flatten(), schema=pyarrow_schema)
+
+    def _get_field_by_id(self, field_id: int) -> Optional[NestedField]:
+        try:
+            return self.file_schema.find_field(field_id)

Review Comment:
   I agree that a lot is happening in the visitor, but since it is just a lookup, I wanted to avoid creating yet another visitor.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +757,68 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ExpressionProjector(BooleanExpressionVisitor[BooleanExpression]):

Review Comment:
   We already use "project" to describe producing an expression for transformed values, so I think it would be confusing to name this "projector". What about something like "name translator"? I think mentioning that it rewrites expression names is a good thing.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +757,68 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ExpressionProjector(BooleanExpressionVisitor[BooleanExpression]):

Review Comment:
   I called it first a translator, and then went with projection :p



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)

Review Comment:
   Yes, this will prune the fields that we don't need, so we only load the column that we're going to need, lowering the pressure on the memory. The magic to match the schema happens in `to_final_schema`, and then we just concat the tables. Since the table is already in the correct format, we can just concat them, with the zero-copy concatenation:
   ![image](https://user-images.githubusercontent.com/1134248/208078066-b715414b-0d2f-4464-9e7b-ee5932a05f28.png)
   
   We could also let PyArrow do the null-filling and promotions, but maybe better to do it ourselves, especially when we start doing things like default values.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool

Review Comment:
   What about `tasks` rather than `files`?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)

Review Comment:
   This is going to produce a subset of the requested schema. If the file has columns a and b, but the requested schema has a, b, and c, then this is going to only have the ones from the file and will produce a dataset with a missing column. That's okay if Arrow knows how to handle it below in `concat_tables`, I think.
   
   We just need to be careful that we are producing an Arrow table that matches the requested schema, not just the file schemas.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",

Review Comment:
   The location would be a base path, not a data file 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyproject.toml:
##########
@@ -222,5 +222,9 @@ ignore_missing_imports = true
 module = "pyparsing.*"
 ignore_missing_imports = true
 
+[[tool.mypy.overrides]]
+module = "numpy.*"
+ignore_missing_imports = true

Review Comment:
   Why is numpy used directly?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")
+
+
+@promote.register(IntegerType)
+def _(file_type: IntegerType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, LongType):
+        # Ints/Longs are binary compatible in Avro, so this is okay
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an int to {read_type}")
+
+
+@promote.register(FloatType)
+def _(file_type: FloatType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DoubleType):
+        # A double type is wider
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an float to {read_type}")
+
+
+@promote.register(StringType)
+def _(file_type: StringType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, BinaryType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an string to {read_type}")
+
+
+@promote.register(BinaryType)
+def _(file_type: BinaryType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, StringType):
+        return read_type
+    else:
+        raise ResolveException(f"Cannot promote an binary to {read_type}")
+
+
+@promote.register(DecimalType)
+def _(file_type: DecimalType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, DecimalType):
+        if file_type.precision <= read_type.precision and file_type.scale == file_type.scale:
+            return read_type
+        else:
+            raise ResolveException(f"Cannot reduce precision from {file_type} to {read_type}")
+    else:
+        raise ResolveException(f"Cannot promote an decimal to {read_type}")
+
+
+@promote.register(StructType)
+def _(file_type: StructType, read_type: IcebergType) -> IcebergType:

Review Comment:
   Yes, we need this because we call it from the `field` method in the visitor, and a field also can have a StructType. We don't have enough information to call this on the primitive method.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")

Review Comment:
   Works for me, updated 👍🏻 



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/avro/test_resolver.py:
##########
@@ -164,17 +163,17 @@ def test_resolver_change_type() -> None:
 
 
 def test_promote_int_to_long() -> None:
-    assert promote(IntegerType(), LongType()) == IntegerReader()
+    assert promote(IntegerType(), LongType()) == LongType()

Review Comment:
   For those tests (and since it is in Avro), we want to use `resolve`. The difference is that with Resolve we check if the promotion is valid, but if we're going from float to double, we still want to read the float (since double is twice as many bytes), so we'll return a float.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,89 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case of an UnboundPredicate
+        ValueError: When a column name cannot be found
+    """
+
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, file_schema: Schema, case_sensitive: bool) -> None:
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        raise TypeError(f"Expected Bound Predicate, got: {predicate.term}")
+
+    def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> BooleanExpression:
+        file_column_name = self.file_schema.find_column_name(predicate.term.ref().field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")
+
+        if isinstance(predicate, BoundUnaryPredicate):
+            return predicate.as_unbound(file_column_name)
+        elif isinstance(predicate, BoundLiteralPredicate):
+            return predicate.as_unbound(file_column_name, predicate.literal)
+        elif isinstance(predicate, BoundSetPredicate):
+            return predicate.as_unbound(file_column_name, predicate.literals)
+        else:
+            raise ValueError(f"Unknown predicate: {predicate}")

Review Comment:
   Unsupported rather than unknown?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,89 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case of an UnboundPredicate
+        ValueError: When a column name cannot be found
+    """
+
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, file_schema: Schema, case_sensitive: bool) -> None:
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        raise TypeError(f"Expected Bound Predicate, got: {predicate.term}")
+
+    def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> BooleanExpression:
+        file_column_name = self.file_schema.find_column_name(predicate.term.ref().field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")

Review Comment:
   Nit: file schema?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")

Review Comment:
   Unsupported type? We can't necessarily conclude that it is a non-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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:

Review Comment:
   Would it make sense to also project the original int column for this table and verify it has the expected values? Otherwise this is nearly identical to the map case in the test 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null

Review Comment:
   The location struct should be optional because it isn't present right?
   
   If a required field is not present in a data file, then that should be an error. We probably want a test case for that.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(

Review Comment:
   In case of promotion, we convert the buffer:
   ```python
   # In case of schema evolution
   if column_arrow_type != expected_arrow_type:
       column_data = column_data.cast(expected_arrow_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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,

Review Comment:
   This isn't very obvious. What about using both `schema_int` and `schema_str` instead? Then you could validate nulls in rows from the file that doesn't have the other column.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)

Review Comment:
   This would be a good test case to add, as well as a test case that uses a predicate on a file that doesn't contain the column, like `col_int > 2` for `table_str`.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(

Review Comment:
   Added! 



##########
python/pyproject.toml:
##########
@@ -222,5 +222,9 @@ ignore_missing_imports = true
 module = "pyparsing.*"
 ignore_missing_imports = true
 
+[[tool.mypy.overrides]]
+module = "numpy.*"
+ignore_missing_imports = true

Review Comment:
   Removed!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null

Review Comment:
   Great suggestion, added!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(

Review Comment:
   Adding more tests, this has far to little tests indeed



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",

Review Comment:
   I've moved this to function to create the `Table`



##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",

Review Comment:
   I've moved this to function to create the `Table` and where this is fixed



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType

Review Comment:
   `requested_schema` and `fiel_schema`?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))

Review Comment:
   This is because we can't select the MapType/ListType:
   ```
                   if not field.field_type.is_primitive:
   >                   raise ValueError(
                           f"Cannot explicitly project List or Map types, {field.field_id}:{field.name} of type {field.field_type} was selected"
                       )
   E                   ValueError: Cannot explicitly project List or Map types, 5:ids of type list<int> was selected
   ```
   We're only interested in the element in case of the list, and the key-value in case of a map.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/types.py:
##########
@@ -268,6 +268,10 @@ def __init__(self, *fields: NestedField, **data: Any):
             data["fields"] = fields
         super().__init__(**data)
 
+    def by_id(self) -> Dict[int, NestedField]:

Review Comment:
   In Java, we use `field(int) -> NestedField` and `field(String) -> NestedField` methods. Can we do that instead to keep a consistent API?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)

Review Comment:
   Auch, normally a type checker would also complain, but the types are the same. Thanks for catching 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] Fokko commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))

Review Comment:
   The question is then if we want to return full structs.
   
   If the requested schema is:
   ```
   location: struct<lat: double>
   ```
   
   Of the table schema:
   ```
   location: struct<lat: double, long: double>
   ```
   
   With the `select_full_types=True` it would return:
   ```
   location: struct<lat: double, long: double>
   ```
   
   Instead of just lat.
   



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,

Review Comment:
   This is tested in `test_projection_filter_add_column`. I've also made this more explicit.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:

Review Comment:
   Good suggestion, updated!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,61 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      table_schema (Schema): The schema of the table

Review Comment:
   Minor: `table_schema` is no longer passed.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(

Review Comment:
   These appear to be the only tests with nested types. I think we want to test that nested structs can also be projected.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[
+                    Schema(
+                        # Reuses the id 1
+                        NestedField(1, "other_id", IntegerType())
+                    )
+                ],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=0,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("other_id", 1),
+        Schema(
+            # Reuses the id 1
+            NestedField(1, "other_id", IntegerType())
+        ),
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 1  # Just 2

Review Comment:
   Comment is out of date. Probably removed a second data 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",

Review Comment:
   This would be a JSON file right?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(

Review Comment:
   I think this is the right approach, although the visitor is going to need to support nested types.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[
+                    Schema(
+                        # Reuses the id 1
+                        NestedField(1, "other_id", IntegerType())
+                    )
+                ],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=0,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("other_id", 1),
+        Schema(
+            # Reuses the id 1
+            NestedField(1, "other_id", IntegerType())
+        ),
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 1  # Just 2

Review Comment:
   Okay, this one was not out of date, it filters `> 1`, so only 2 is in the results



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    struct_array = visit_with_partner(
+        requested_schema, table, ArrowProjectionVisitor(file_schema, len(table)), ArrowAccessor(file_schema)
+    )
+
+    arrays = []
+    fields = []
+    for pos, field in enumerate(requested_schema.fields):
+        array = struct_array.field(pos)
+        arrays.append(array)
+        fields.append(pa.field(field.name, array.type, field.optional))
+    return pa.Table.from_arrays(arrays, schema=pa.schema(fields))
+
+
+class ArrowProjectionVisitor(SchemaWithPartnerVisitor[pa.Array, Optional[pa.Array]]):
+    file_schema: Schema
+    table_length: int
+
+    def __init__(self, file_schema: Schema, table_length: int):
+        self.file_schema = file_schema
+        self.table_length = table_length
+
+    def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
+        file_field = self.file_schema.find_field(field.field_id)
+        if field.field_type != file_field.field_type:

Review Comment:
   This needs to be updated to check whether the expected type is a primitive. Otherwise, when you have nested structs or other complex types, it fails because `promote` doesn't support them.
   
   ```python
       def cast_if_needed(self, field: NestedField, values: pa.Array) -> pa.Array:
           file_field = self.file_schema.find_field(field.field_id)
           if field.field_type.is_primitive and field.field_type != file_field.field_type:
               return values.cast(schema_to_pyarrow(promote(file_field.field_type, field.field_type)))
   ```
   
   I caught this when testing the nested list case.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/exceptions.py:
##########
@@ -86,3 +86,7 @@ class NotInstalledError(Exception):
 
 class SignError(Exception):
     """Raises when unable to sign a S3 request"""
+
+
+class ResolveException(Exception):

Review Comment:
   Good catch, updated 👍🏻 



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/types.py:
##########
@@ -268,6 +268,10 @@ def __init__(self, *fields: NestedField, **data: Any):
             data["fields"] = fields
         super().__init__(**data)
 
+    def by_id(self) -> Dict[int, NestedField]:

Review Comment:
   Good call, updated!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)

Review Comment:
   Minor: we should detect when the Iceberg schema is missing and raise a friendly exception that says the file can't be read because it doesn't have an Iceberg schema.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(

Review Comment:
   What happens when one file has a column but others don't? For example, if I run `ALTER TABLE t ADD COLUMN c int` and read files before and after the change? Also, what happens if we end up with mixed types as well because c was promoted from `int` to `long`?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +757,68 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ExpressionProjector(BooleanExpressionVisitor[BooleanExpression]):
+    """Rewrites a boolean expression by replacing unbound references with references to fields in a struct schema
+
+    Args:
+      table_schema (Schema): The schema of the table
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case a predicate is already bound
+    """
+
+    table_schema: Schema
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, table_schema: Schema, file_schema: Schema, case_sensitive: bool) -> None:
+        self.table_schema = table_schema
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        if not isinstance(predicate.term, Reference):
+            raise ValueError(f"Exprected reference: {predicate.term}")
+
+        field = self.table_schema.find_field(predicate.term.name, case_sensitive=self.case_sensitive)
+        file_column_name = self.file_schema.find_column_name(field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")
+
+        if isinstance(predicate, UnaryPredicate):
+            return predicate.__class__(Reference(file_column_name))

Review Comment:
   Ah, I see. I think it is nicer to convert it back to an unbound expression, and then bind that with the file schema. Let me update the 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] Fokko commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:

Review Comment:
   I think it would make more sense to make this part of the task, but lets do that in a separate PR



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(

Review Comment:
   You may want a factory method to create metadata for these tests. That would cut down on a lot of duplicate lines.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[
+                    Schema(

Review Comment:
   This schema is used twice. Maybe use a variable for 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None

Review Comment:
   Do we need to assert that there are 3 `None` values in each column to correspond to the 3 rows in the 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:

Review Comment:
   I don't think it is very clear what "final" means in this context. We usually refer to this as the "requested" or "expected" schema. We may want to rename to use that convention.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:

Review Comment:
   This approach assumes that the column data is available at a specific column index in the table. I'm not sure that is true. It _might_ be if the column index space is flat (e.g. id, location.lat, location.lon, data) but it may not be (e.g. id, location, data).
   
   If that's the case, you will probably want a [`SchemaWithPartnerVisitor`](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/schema/SchemaWithPartnerVisitor.java) to traverse two structures (the table and the schema) at the same 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] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:
+        """Visit a Schema with a concrete SchemaVisitorWithPartner"""
+        struct_result = self.visit_with_arrow(requested_schema.as_struct(), file_schema.as_struct())
+        pyarrow_schema = schema_to_pyarrow(requested_schema)
+        return pa.Table.from_arrays(struct_result.flatten(), schema=pyarrow_schema)
+
+    def _get_field_by_id(self, field_id: int) -> Optional[NestedField]:
+        try:
+            return self.file_schema.find_field(field_id)

Review Comment:
   Visitors should not use `find_*` methods. Those methods will return columns from anywhere in a schema. It looks like this is used below to find a field in the file schema rather than getting the field from `file_struct`. I think it should look up the field in the struct rather than doing this.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/avro/test_resolver.py:
##########
@@ -164,17 +163,17 @@ def test_resolver_change_type() -> None:
 
 
 def test_promote_int_to_long() -> None:
-    assert promote(IntegerType(), LongType()) == IntegerReader()
+    assert promote(IntegerType(), LongType()) == LongType()

Review Comment:
   Is it better to test `promote` or `resolve` here? I don't think it matters much.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,89 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case of an UnboundPredicate
+        ValueError: When a column name cannot be found
+    """
+
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, file_schema: Schema, case_sensitive: bool) -> None:
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        raise TypeError(f"Expected Bound Predicate, got: {predicate.term}")
+
+    def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> BooleanExpression:
+        file_column_name = self.file_schema.find_column_name(predicate.term.ref().field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")
+
+        if isinstance(predicate, BoundUnaryPredicate):
+            return predicate.as_unbound(file_column_name)
+        elif isinstance(predicate, BoundLiteralPredicate):
+            return predicate.as_unbound(file_column_name, predicate.literal)
+        elif isinstance(predicate, BoundSetPredicate):
+            return predicate.as_unbound(file_column_name, predicate.literals)
+        else:
+            raise ValueError(f"Unknown predicate: {predicate}")

Review Comment:
   Good call



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,89 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case of an UnboundPredicate
+        ValueError: When a column name cannot be found
+    """
+
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, file_schema: Schema, case_sensitive: bool) -> None:
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        raise TypeError(f"Expected Bound Predicate, got: {predicate.term}")
+
+    def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> BooleanExpression:
+        file_column_name = self.file_schema.find_column_name(predicate.term.ref().field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")

Review Comment:
   Great suggestion



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done

Review Comment:
   `ResolveError`



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(

Review Comment:
   I left this out initially because I recall that we don't support schema evolution for nested types. 



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -44,8 +49,14 @@
     S3FileSystem,
 )
 
-from pyiceberg.expressions import BooleanExpression, BoundTerm, Literal
-from pyiceberg.expressions.visitors import BoundBooleanExpressionVisitor
+from pyiceberg.avro.resolver import promote

Review Comment:
   I've split this out into two visitors, one solely for promoting types, and one for converting it into a reader



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:

Review Comment:
   Ah nice, I was missing this piece.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,

Review Comment:
   I've removed all of them, and let them default to zero



-- 
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 merged pull request #6437: Python: Projection by Field ID

Posted by GitBox <gi...@apache.org>.
Fokko merged PR #6437:
URL: https://github.com/apache/iceberg/pull/6437


-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)

Review Comment:
   That's a good question. Currently it throws:
   ```
   _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
   test_pyarrow.py:740: in project
       return project_table(
   ../../pyiceberg/io/pyarrow.py:495: in project_table
       bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
   ../../pyiceberg/expressions/visitors.py:203: in bind
       return visit(expression, BindVisitor(schema, case_sensitive))
   /Library/Developer/CommandLineTools/Library/Frameworks/Python3.framework/Versions/3.9/lib/python3.9/functools.py:877: in wrapper
       return dispatch(args[0].__class__)(*args, **kw)
   ../../pyiceberg/expressions/visitors.py:175: in _
       return visitor.visit_unbound_predicate(predicate=obj)
   ../../pyiceberg/expressions/visitors.py:240: in visit_unbound_predicate
       return predicate.bind(self.schema, case_sensitive=self.case_sensitive)
   ../../pyiceberg/expressions/__init__.py:615: in bind
       bound_term = self.term.bind(schema, case_sensitive)
   ../../pyiceberg/expressions/__init__.py:180: in bind
       field = schema.find_field(name_or_id=self.name, case_sensitive=case_sensitive)
   _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
   
   self = Schema(NestedField(field_id=1, name='id', field_type=IntegerType(), required=False), NestedField(field_id=2, name='data', field_type=StringType(), required=False), schema_id=0, identifier_field_ids=[])
   name_or_id = 'unknown_field', case_sensitive = True
   
       def find_field(self, name_or_id: Union[str, int], case_sensitive: bool = True) -> NestedField:
           """Find a field using a field name or field ID
       
           Args:
               name_or_id (str | int): Either a field name or a field ID
               case_sensitive (bool, optional): Whether to perform a case-sensitive lookup using a field name. Defaults to True.
       
           Raises:
               ValueError: When the value cannot be found
       
           Returns:
               NestedField: The matched NestedField
           """
           if isinstance(name_or_id, int):
               if name_or_id not in self._lazy_id_to_field:
                   raise ValueError(f"Could not find field with id: {name_or_id}")
               return self._lazy_id_to_field[name_or_id]
       
           if case_sensitive:
               field_id = self._name_to_id.get(name_or_id)
           else:
               field_id = self._lazy_name_to_id_lower.get(name_or_id.lower())
       
           if field_id is None:
   >           raise ValueError(f"Could not find field with name {name_or_id}, case_sensitive={case_sensitive}")
   E           ValueError: Could not find field with name unknown_field, case_sensitive=True
   ```
   Also embedded this in a test-case



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +756,89 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]):
+    """Converts the column names with the ones in the actual file
+
+    Args:
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case of an UnboundPredicate
+        ValueError: When a column name cannot be found
+    """
+
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, file_schema: Schema, case_sensitive: bool) -> None:
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        raise TypeError(f"Expected Bound Predicate, got: {predicate.term}")
+
+    def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> BooleanExpression:
+        file_column_name = self.file_schema.find_column_name(predicate.term.ref().field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")
+
+        if isinstance(predicate, BoundUnaryPredicate):
+            return predicate.as_unbound(file_column_name)
+        elif isinstance(predicate, BoundLiteralPredicate):
+            return predicate.as_unbound(file_column_name, predicate.literal)
+        elif isinstance(predicate, BoundSetPredicate):
+            return predicate.as_unbound(file_column_name, predicate.literals)
+        else:
+            raise ValueError(f"Unknown predicate: {predicate}")
+
+
+def translate_column_names(expr: BooleanExpression, file_schema: Schema, case_sensitive: bool) -> BooleanExpression:
+    return visit(expr, _ColumnNameTranslator(file_schema, case_sensitive))
+
+
+class ExpressionFieldIDs(BooleanExpressionVisitor[Set[int]]):

Review Comment:
   In some cases, we prefix the visitor classes with `_`, but I think it's really easy to forget to do it so we're inconsistent. Is there a good way to signal that people shouldn't rely on the API or behavior of these classes? I'm also fine with just not worrying about it and using normal names.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))

Review Comment:
   I think this is related to the use of `select_full_types=False` below, but that flag is based on the logic in the Java implementation's `GetProjectedIds`. That visitor accumulates a list of field IDs throughout the schema that are projected, although it doesn't include map or list fields because the map key/value and list element fields are directly selected.
   
   If this is going to use `select_full_types=False` then it should apply the same visitor logic and set the visitor that returns projected IDs to include struct IDs.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))

Review Comment:
   Why does this exclude maps and lists?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:
+        """Visit a Schema with a concrete SchemaVisitorWithPartner"""
+        struct_result = self.visit_with_arrow(requested_schema.as_struct(), file_schema.as_struct())
+        pyarrow_schema = schema_to_pyarrow(requested_schema)
+        return pa.Table.from_arrays(struct_result.flatten(), schema=pyarrow_schema)
+
+    def _get_field_by_id(self, field_id: int) -> Optional[NestedField]:
+        try:
+            return self.file_schema.find_field(field_id)
+        except ValueError:
+            # Field is not in the file
+            return None
+
+    @visit_with_arrow.register(StructType)
+    def _(self, requested_struct: StructType, file_struct: Optional[IcebergType]) -> pa.Array:  # pylint: disable=unused-argument
+        """Visit a StructType with a concrete SchemaVisitorWithPartner"""
+        results = []
+
+        for requested_field in requested_struct.fields:
+            file_field = self._get_field_by_id(requested_field.field_id)
+
+            if file_field is None and requested_field.required:
+                raise ResolveException(f"Field is required, and could not be found in the file: {requested_field}")
+
+            results.append(self.visit_with_arrow(requested_field.field_type, file_field))
+
+        pyarrow_schema = schema_to_pyarrow(requested_struct)
+        return pa.StructArray.from_arrays(arrays=results, fields=pyarrow_schema)
+
+    @visit_with_arrow.register(ListType)
+    def _(self, requested_list: ListType, file_field: Optional[NestedField]) -> pa.Array:
+        """Visit a ListType with a concrete SchemaVisitorWithPartner"""
+
+        if file_field is not None:
+            if not isinstance(file_field.field_type, ListType):
+                raise ValueError(f"Expected list, got: {file_field}")
+
+            return self.visit_with_arrow(requested_list.element_type, self._get_field_by_id(file_field.field_type.element_id))
+        else:
+            # Not in the file, fill in with nulls
+            return pa.nulls(len(self.table), type=pa.list_(schema_to_pyarrow(requested_list.element_type)))
+
+    @visit_with_arrow.register(MapType)
+    def _(self, requested_map: MapType, file_map: Optional[NestedField]) -> pa.Array:
+        """Visit a MapType with a concrete SchemaVisitorWithPartner"""
+
+        if file_map is not None:
+            if not isinstance(file_map.field_type, MapType):
+                raise ValueError(f"Expected map, got: {file_map}")
+
+            key = self._get_field_by_id(file_map.field_type.key_id)
+            return self.visit_with_arrow(requested_map.key_type, key)
+        else:
+            # Not in the file, fill in with nulls
+            return pa.nulls(
+                len(self.table),
+                type=pa.map_(schema_to_pyarrow(requested_map.key_type), schema_to_pyarrow(requested_map.value_type)),
+            )
+
+    def _get_column_data(self, file_field: NestedField) -> pa.Array:
+        column_name = self.file_schema.find_column_name(file_field.field_id)
+        column_data = self.table
+        struct_schema = self.table.schema
+
+        if column_name is None:
+            # Should not happen
+            raise ValueError(f"Could not find column: {column_name}")
+
+        column_parts = list(reversed(column_name.split(".")))
+        while len(column_parts) > 1:
+            part = column_parts.pop()
+            column_data = column_data.column(part)
+            struct_schema = struct_schema[struct_schema.get_field_index(part)].type
+
+        if not isinstance(struct_schema, (pa.ListType, pa.MapType)):
+            # PyArrow does not have an element
+            idx = struct_schema.get_field_index(column_parts.pop())
+            column_data = column_data.flatten()[idx]
+
+        return column_data.combine_chunks()

Review Comment:
   I have the same concern. I think we should be able to avoid this using the lower-level buffer API, but still have to dig into that. I wanted to get it to work first, and then we can make it fast :)



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/schema.py:
##########
@@ -1046,3 +1055,79 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType:
                 value_type=value_result,
                 value_required=map_type.value_required,
             )
+
+
+@singledispatch
+def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType:
+    """Promotes reading a file type to a read type
+
+    Args:
+        file_type (IcebergType): The type of the Avro file
+        read_type (IcebergType): The requested read type
+
+    Raises:
+        ResolveException: If attempting to resolve an unrecognized object type
+    """
+    raise ResolveException(f"Cannot promote {file_type} to {read_type}")
+
+
+@promote.register(IntegerType)
+def _(file_type: IntegerType, read_type: IcebergType) -> IcebergType:
+    if isinstance(read_type, LongType):

Review Comment:
   The promote function is only called when the read and file are different.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +457,103 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = project_expression(row_filter, table.schema(), file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Iceberg schema not encoded in Parquet file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(

Review Comment:
   I've added tests for both of them. If a column isn't there, we create a PyArrow buffer, filled with `null`s. I noticed that in [Python legacy](https://github.com/apache/iceberg/blob/master/python_legacy/iceberg/parquet/parquet_reader.py#L43-L67), we filled with `{}`, `[]`, `np.NaN` etc, but that makes less sense to me.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:

Review Comment:
   Can you add a case like this where a subset of rows match the filter?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)

Review Comment:
   Does PyArrow handle filters that are not in the requested projection? For example, if I run `SELECT event_type FROM events WHERE event_ts > DATE '2022-12-18'` will Arrow correctly filter by `event_ts` but return a dataset with only `event_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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)

Review Comment:
   I think numpy uses `NaN` for null values. Is there a direct Arrow way to do this without using numpy?



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass

Review Comment:
   I think we will need to implement at least `map` and `list`.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:

Review Comment:
   Fair point



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +465,198 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)
+
+        arrow_table = ds.dataset(
+            source=[path], schema=file_project_schema_arrow, format=ds.ParquetFileFormat(), filesystem=fs
+        ).to_table(filter=pyarrow_filter)
+
+        tables.append(to_requested_schema(projected_schema, file_project_schema, arrow_table))
+
+    if len(tables) > 1:
+        return pa.concat_tables(tables)
+    else:
+        return tables[0]
+
+
+def to_requested_schema(requested_schema: Schema, file_schema: Schema, table: pa.Table) -> pa.Table:
+    return VisitWithArrow(requested_schema, file_schema, table).visit()
+
+
+class VisitWithArrow:
+    requested_schema: Schema
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, requested_schema: Schema, file_schema: Schema, table: pa.Table) -> None:
+        self.requested_schema = requested_schema
+        self.file_schema = file_schema
+        self.table = table
+
+    def visit(self) -> pa.Table:
+        return self.visit_with_arrow(self.requested_schema, self.file_schema)
+
+    @singledispatchmethod
+    def visit_with_arrow(self, requested_schema: Union[Schema, IcebergType], file_schema: Union[Schema, IcebergType]) -> pa.Table:
+        """A generic function for applying a schema visitor to any point within a schema
+
+        The function traverses the schema in post-order fashion
+
+        Args:
+            obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+            visitor (VisitWithArrow[T]): An instance of an implementation of the generic VisitWithArrow base class
+
+        Raises:
+            NotImplementedError: If attempting to visit an unrecognized object type
+        """
+        raise NotImplementedError(f"Cannot visit non-type: {requested_schema}")
+
+    @visit_with_arrow.register(Schema)
+    def _(self, requested_schema: Schema, file_schema: Schema) -> pa.Table:
+        """Visit a Schema with a concrete SchemaVisitorWithPartner"""
+        struct_result = self.visit_with_arrow(requested_schema.as_struct(), file_schema.as_struct())
+        pyarrow_schema = schema_to_pyarrow(requested_schema)
+        return pa.Table.from_arrays(struct_result.flatten(), schema=pyarrow_schema)

Review Comment:
   It looks like this is both a custom schema visitor and part visitor implementation?
   
   In general, we avoid mixing logic to traverse structures with the functional logic of what to do with the structure.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)
+        return column_data
+
+    def list(self, _: ListType, element_result: pa.ChunkedArray) -> pa.ChunkedArray:
+        pass
+
+    def map(self, _: MapType, key_result: pa.ChunkedArray, value_result: pa.ChunkedArray) -> pa.DataType:
+        pass
+
+    def primitive(self, primitive: PrimitiveType) -> pa.ChunkedArray:
+        pass
+
+
+def to_final_schema(final_schema: Schema, schema: Schema, table: pa.Table) -> pa.Table:
+    return visit(final_schema, _ConstructFinalSchema(schema, table))
+
+
+def project_table(
+    files: Iterable["FileScanTask"], table: "Table", row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    projected_field_ids = projected_schema.field_ids
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        file_project_schema = prune_columns(file_schema, projected_field_ids)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)
+
+        if file_schema is None:
+            raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
+
+        # Prune the stuff that we don't need anyway
+        file_project_schema_arrow = schema_to_pyarrow(file_project_schema)

Review Comment:
   This would be a good test case to add.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +459,120 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+class _ConstructFinalSchema(SchemaVisitor[pa.ChunkedArray]):
+    file_schema: Schema
+    table: pa.Table
+
+    def __init__(self, file_schema: Schema, table: pa.Table):
+        self.file_schema = file_schema
+        self.table = table
+
+    def schema(self, schema: Schema, struct_result: List[pa.ChunkedArray]) -> pa.Table:
+        return pa.table(struct_result, schema=schema_to_pyarrow(schema))
+
+    def struct(self, _: StructType, field_results: List[pa.ChunkedArray]) -> List[pa.ChunkedArray]:
+        return field_results
+
+    def field(self, field: NestedField, _: pa.ChunkedArray) -> pa.ChunkedArray:
+        column_name = self.file_schema.find_column_name(field.field_id)
+
+        if column_name:
+            column_idx = self.table.schema.get_field_index(column_name)
+        else:
+            column_idx = -1
+
+        expected_arrow_type = schema_to_pyarrow(field.field_type)
+
+        # The idx will be -1 when the column can't be found
+        if column_idx >= 0:
+            column_field: pa.Field = self.table.schema[column_idx]
+            column_arrow_type: pa.DataType = column_field.type
+            column_data: pa.ChunkedArray = self.table[column_idx]
+            file_type = self.file_schema.find_type(field.field_id)
+
+            # In case of schema evolution
+            if column_arrow_type != expected_arrow_type:
+                # To check if the promotion is allowed
+                _ = promote(file_type, field.field_type)
+                column_data = column_data.cast(expected_arrow_type)
+        else:
+            import numpy as np
+
+            column_data = pa.array(np.full(shape=len(self.table), fill_value=None), type=expected_arrow_type)

Review Comment:
   https://arrow.apache.org/docs/python/generated/pyarrow.nulls.html



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -572,3 +581,388 @@ def test_always_true_to_pyarrow(bound_reference: BoundReference[str]) -> None:
 
 def test_always_false_to_pyarrow(bound_reference: BoundReference[str]) -> None:
     assert repr(expression_to_pyarrow(AlwaysFalse())) == "<pyarrow.compute.Expression false>"
+
+
+@pytest.fixture
+def schema_int() -> Schema:
+    return Schema(NestedField(1, "id", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_str() -> Schema:
+    return Schema(NestedField(2, "data", IntegerType()), schema_id=1)
+
+
+@pytest.fixture
+def schema_long() -> Schema:
+    return Schema(NestedField(3, "id", LongType()), schema_id=1)
+
+
+@pytest.fixture
+def table_int(schema_int: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_int), metadata={"iceberg.schema": schema_int.json()})
+
+    target_file = f"file:{tmpdir}/a.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_str(schema_str: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_str), metadata={"iceberg.schema": schema_str.json()})
+
+    target_file = f"file:{tmpdir}/b.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+@pytest.fixture
+def table_long(schema_long: Schema, tmpdir: str) -> str:
+    pyarrow_schema = pa.schema(schema_to_pyarrow(schema_long), metadata={"iceberg.schema": schema_long.json()})
+
+    target_file = f"file:{tmpdir}/c.parquet"
+
+    with pq.ParquetWriter(target_file, pyarrow_schema) as writer:
+        writer.write_table(pa.Table.from_arrays([pa.array([0, 1, 2])], schema=pyarrow_schema))
+
+    return target_file
+
+
+def test_projection_add_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # All new IDs
+        NestedField(10, "id", IntegerType(), required=False),
+        NestedField(20, "list", ListType(21, IntegerType(), element_required=False), required=False),
+        NestedField(
+            30,
+            "map",
+            MapType(key_id=31, key_type=IntegerType(), value_id=32, value_type=StringType(), value_required=False),
+            required=False,
+        ),
+        NestedField(40, "location", StructType(NestedField(41, "lat", DoubleType()), NestedField(42, "lon", DoubleType()))),
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+
+    # Everything should be None
+    for col in result_table.columns:
+        for r in col:
+            assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """id: int32
+list: list<item: int32>
+  child 0, item: int32
+map: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string
+location: struct<lat: double not null, lon: double not null> not null
+  child 0, lat: double not null
+  child 1, lon: double not null"""
+    )
+
+
+def test_projection_add_column_struct(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # A new ID
+        NestedField(
+            2,
+            "other_id",
+            MapType(key_id=3, key_type=IntegerType(), value_id=4, value_type=StringType(), value_required=False),
+            required=False,
+        )
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    # Everything should be None
+    for r in result_table.columns[0]:
+        assert r.as_py() is None
+
+    assert (
+        repr(result_table.schema)
+        == """other_id: map<int32, string>
+  child 0, entries: struct<key: int32 not null, value: string> not null
+      child 0, key: int32 not null
+      child 1, value: string"""
+    )
+
+
+def test_projection_rename_column(schema_int: Schema, table_int: str) -> None:
+    schema = Schema(
+        # Reuses the id 1
+        NestedField(1, "other_id", IntegerType())
+    )
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema],
+                partition_specs=[PartitionSpec()],
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx
+
+    assert repr(result_table.schema) == "other_id: int32 not null"
+
+
+def test_projection_concat_files(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        AlwaysTrue(),
+        schema_int,
+        case_sensitive=True,
+    )
+    for idx, r in enumerate(result_table.columns[0]):
+        assert r.as_py() == idx % 3
+    assert len(result_table.columns[0]) == 6
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter(schema_int: Schema, table_int: str) -> None:
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            ),
+        ]
+        * 2,
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[schema_int],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=1,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("id", 4),
+        schema_int,
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 0
+    assert repr(result_table.schema) == "id: int32 not null"
+
+
+def test_projection_filter_renamed_column(schema_int: Schema, table_int: str) -> None:
+    """Filter on a renamed column"""
+    result_table = project_table(
+        [
+            FileScanTask(
+                DataFile(file_path=table_int, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3)
+            )
+        ],
+        Table(
+            ("namespace", "table"),
+            metadata=TableMetadataV2(
+                location="file://a/b/c.parquet",
+                last_column_id=1,
+                format_version=2,
+                schemas=[
+                    Schema(
+                        # Reuses the id 1
+                        NestedField(1, "other_id", IntegerType())
+                    )
+                ],
+                partition_specs=[PartitionSpec()],
+                current_schema_id=0,
+            ),
+            metadata_location="file://a/b/c.parquet",
+            io=PyArrowFileIO(),
+        ),
+        GreaterThan("other_id", 1),
+        Schema(
+            # Reuses the id 1
+            NestedField(1, "other_id", IntegerType())
+        ),
+        case_sensitive=True,
+    )
+    assert len(result_table.columns[0]) == 1  # Just 2

Review Comment:
   Exactly!



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/expressions/visitors.py:
##########
@@ -753,3 +757,68 @@ def inclusive_projection(
     schema: Schema, spec: PartitionSpec, case_sensitive: bool = True
 ) -> Callable[[BooleanExpression], BooleanExpression]:
     return InclusiveProjection(schema, spec, case_sensitive).project
+
+
+class _ExpressionProjector(BooleanExpressionVisitor[BooleanExpression]):
+    """Rewrites a boolean expression by replacing unbound references with references to fields in a struct schema
+
+    Args:
+      table_schema (Schema): The schema of the table
+      file_schema (Schema): The schema of the file
+      case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True
+
+    Raises:
+        TypeError: In the case a predicate is already bound
+    """
+
+    table_schema: Schema
+    file_schema: Schema
+    case_sensitive: bool
+
+    def __init__(self, table_schema: Schema, file_schema: Schema, case_sensitive: bool) -> None:
+        self.table_schema = table_schema
+        self.file_schema = file_schema
+        self.case_sensitive = case_sensitive
+
+    def visit_true(self) -> BooleanExpression:
+        return AlwaysTrue()
+
+    def visit_false(self) -> BooleanExpression:
+        return AlwaysFalse()
+
+    def visit_not(self, child_result: BooleanExpression) -> BooleanExpression:
+        return Not(child=child_result)
+
+    def visit_and(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return And(left=left_result, right=right_result)
+
+    def visit_or(self, left_result: BooleanExpression, right_result: BooleanExpression) -> BooleanExpression:
+        return Or(left=left_result, right=right_result)
+
+    def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> BooleanExpression:
+        if not isinstance(predicate.term, Reference):
+            raise ValueError(f"Exprected reference: {predicate.term}")
+
+        field = self.table_schema.find_field(predicate.term.name, case_sensitive=self.case_sensitive)
+        file_column_name = self.file_schema.find_column_name(field.field_id)
+
+        if not file_column_name:
+            raise ValueError(f"Not found in schema: {file_column_name}")
+
+        if isinstance(predicate, UnaryPredicate):
+            return predicate.__class__(Reference(file_column_name))

Review Comment:
   Shouldn't this be `predicate.as_unbound(Reference(file_column_name))`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)

Review Comment:
   Looks like the name `bound_row_filter` is reused, which is going to cause problems because it leaks the filter for the last iteration into the current iteration. Probably best to use `bound_file_filter` or something.



-- 
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 #6437: Python: Projection by Field ID

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -437,3 +468,170 @@ def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> p
 
 def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
+
+
+def project_table(
+    files: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool
+) -> pa.Table:
+    """Resolves the right columns based on the identifier
+
+    Args:
+        files(Iterable[FileScanTask]): A URI or a path to a local file
+        table(Table): The table that's being queried
+        row_filter(BooleanExpression): The expression for filtering rows
+        projected_schema(Schema): The output schema
+        case_sensitive(bool): Case sensitivity when looking up column names
+
+    Raises:
+        ResolveException: When an incompatible query is done
+    """
+
+    if isinstance(table.io, PyArrowFileIO):
+        scheme, path = PyArrowFileIO.parse_location(table.location())
+        fs = table.io.get_fs(scheme)
+    else:
+        raise ValueError(f"Expected PyArrowFileIO, got: {table.io}")
+
+    bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive)
+
+    projected_field_ids = {
+        id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
+    }.union(extract_field_ids(bound_row_filter))
+
+    tables = []
+    for task in files:
+        _, path = PyArrowFileIO.parse_location(task.file.file_path)
+
+        # Get the schema
+        with fs.open_input_file(path) as fout:
+            parquet_schema = pq.read_schema(fout)
+            schema_raw = parquet_schema.metadata.get(ICEBERG_SCHEMA)
+            file_schema = Schema.parse_raw(schema_raw)
+
+        pyarrow_filter = None
+        if row_filter is not AlwaysTrue():
+            translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
+            bound_row_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
+            pyarrow_filter = expression_to_pyarrow(bound_row_filter)

Review Comment:
   For cases where the file does not contain a column that is being filtered, does binding to the file schema throw an exception?



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