rymurr commented on a change in pull request #1727:
URL: https://github.com/apache/iceberg/pull/1727#discussion_r519833436



##########
File path: python/iceberg/parquet/dataset_utils.py
##########
@@ -0,0 +1,177 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from iceberg.api.expressions import Expression, Operation, Predicate
+import pyarrow.dataset as ds
+
+
+def get_dataset_filter(expr: Expression, expected_to_file_map: dict) -> 
ds.Expression:

Review comment:
       just curious why we have `get_dataset_filter` and `get_expr`. Maybe it 
will become clear as I review the rest but from here it seems redundnat

##########
File path: python/iceberg/core/util/profile.py
##########
@@ -0,0 +1,36 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from contextlib import contextmanager
+import logging
+import time
+
+_logger = logging.getLogger(__name__)
+
+
+@contextmanager
+def profile(label, stats_dict=None):
+    if stats_dict is None:
+        print('PROFILE: %s starting' % label)

Review comment:
       probably want this to be _logger.debug

##########
File path: python/iceberg/parquet/parquet_reader.py
##########
@@ -0,0 +1,240 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from datetime import datetime
+import decimal
+import logging
+from typing import Any, Callable, Dict, List, Tuple, Union
+
+from iceberg.api import Schema
+from iceberg.api.expressions import Expression
+from iceberg.api.io import InputFile
+from iceberg.api.types import NestedField, Type, TypeID
+from iceberg.core.util.profile import profile
+from iceberg.exceptions import InvalidCastException
+import numpy as np
+import pandas as pd
+import pyarrow as pa
+from pyarrow import fs
+import pyarrow.dataset as ds
+import pyarrow.parquet as pq
+
+from .dataset_utils import get_dataset_filter
+from .parquet_schema_utils import prune_columns
+from .parquet_to_iceberg import convert_parquet_to_iceberg
+
+_logger = logging.getLogger(__name__)
+
+DTYPE_MAP: Dict[TypeID,
+                Callable[[NestedField], Tuple[pa.Field, Any]]] = \
+    {TypeID.BINARY: lambda field: pa.binary(),
+     TypeID.BOOLEAN: lambda field: (pa.bool_(), False),
+     TypeID.DATE: lambda field: (pa.date32(), datetime.now()),
+     TypeID.DECIMAL: lambda field: (pa.decimal128(field.type.precision, 
field.type.scale),
+                                    decimal.Decimal()),
+     TypeID.DOUBLE: lambda field: (pa.float64(), np.nan),
+     TypeID.FIXED: lambda field: pa.binary(field.length),
+     TypeID.FLOAT: lambda field: (pa.float32(), np.nan),
+     TypeID.INTEGER: lambda field: (pa.int32(), np.nan),
+     TypeID.LIST: lambda field: (pa.list_(pa.field("element",
+                                                   
DTYPE_MAP[field.type.element_type.type_id](field.type)[0])),
+                                 None),
+     TypeID.LONG: lambda field: (pa.int64(), np.nan),
+     # To-Do: update to support reading map fields
+     # TypeID.MAP: lambda field: (,),
+     TypeID.STRING: lambda field: (pa.string(), ""),
+     TypeID.STRUCT: lambda field: (pa.struct([(nested_field.name,
+                                               
DTYPE_MAP[nested_field.type.type_id](nested_field.type)[0])
+                                              for nested_field in 
field.type.fields]), {}),
+     TypeID.TIMESTAMP: lambda field: (pa.timestamp("us"), datetime.now()),
+     # not used in SPARK, so not implementing for now
+     # TypeID.TIME: pa.time64(None)
+     }
+
+
+class ParquetReader(object):
+
+    def __init__(self, input: InputFile, expected_schema: Schema, options, 
filter_expr: Expression,
+                 case_sensitive: bool, start: int = None, end: int = None):
+        self._stats: Dict[str, int] = dict()
+
+        self._input = input
+        self._input_fo = input.new_fo()
+
+        self._arrow_file = pq.ParquetFile(self._input_fo)
+        self._file_schema = convert_parquet_to_iceberg(self._arrow_file)
+        self._expected_schema = expected_schema
+        self._file_to_expected_name_map = 
ParquetReader.get_field_map(self._file_schema,
+                                                                      
self._expected_schema)
+        self._options = options
+        self._filter = get_dataset_filter(filter_expr, 
ParquetReader.get_reverse_field_map(self._file_schema,
+                                                                               
            self._expected_schema))
+
+        self._case_sensitive = case_sensitive
+        if start is not None or end is not None:
+            raise NotImplementedError("Partial file reads are not yet 
supported")
+            # self.start = start
+            # self.end = end
+
+        self.materialized_table = False
+        self.curr_iterator = None
+        self._table = None
+        self._df = None
+
+        _logger.debug("Reader initialized for %s" % self._input.path)
+
+    @property
+    def stats(self) -> dict:
+        return dict(self._stats)
+
+    def to_pandas(self) -> Union[pd.Series, pd.DataFrame]:
+        if not self.materialized_table:
+            self._read_data()
+            with profile("to_pandas", self._stats):
+                if self._table is not None:
+                    self._df = self._table.to_pandas(use_threads=True)
+                else:
+                    self._df = None
+
+        return self._df
+
+    def to_arrow_table(self) -> pa.Table:
+        if not self.materialized_table:
+            self._read_data()
+
+        return self._table
+
+    def _read_data(self) -> None:
+        _logger.debug("Starting data read")
+
+        # only scan the columns projected and in our file
+        cols_to_read = prune_columns(self._file_schema, self._expected_schema)
+
+        with profile("read data", self._stats):
+            arrow_dataset = 
ds.FileSystemDataset.from_paths([self._input.location()],

Review comment:
       Its a shame that we can't do this for a batch of files. Seems like a 
common pattern that pyarrow should be able to handle. Have you asked on 
[email protected] if there is a plan to support this?
   
   Do you have a sense of the performance hit of reading files individually as 
opposed to in one batch? I am worried it could be significant. 
   
   Finally, what is the path for reading/iterating multi-file iceberg tables? 
Do we read 1 parrquet file at a time or all at once or all at once as a pandas 
df? One of the key features of Datasets (for me) was being able to control 
memory when doing large reads.

##########
File path: python/setup.py
##########
@@ -38,7 +37,7 @@
                       'requests',
                       'retrying',
                       'pandas',
-                      'pyarrow'
+                      'pyarrow>=0.17.0'

Review comment:
       why not pyarrow 2.0.0?

##########
File path: python/iceberg/parquet/parquet_reader.py
##########
@@ -0,0 +1,240 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from datetime import datetime
+import decimal
+import logging
+from typing import Any, Callable, Dict, List, Tuple, Union
+
+from iceberg.api import Schema
+from iceberg.api.expressions import Expression
+from iceberg.api.io import InputFile
+from iceberg.api.types import NestedField, Type, TypeID
+from iceberg.core.util.profile import profile
+from iceberg.exceptions import InvalidCastException
+import numpy as np
+import pandas as pd
+import pyarrow as pa
+from pyarrow import fs
+import pyarrow.dataset as ds
+import pyarrow.parquet as pq
+
+from .dataset_utils import get_dataset_filter
+from .parquet_schema_utils import prune_columns
+from .parquet_to_iceberg import convert_parquet_to_iceberg
+
+_logger = logging.getLogger(__name__)
+
+DTYPE_MAP: Dict[TypeID,
+                Callable[[NestedField], Tuple[pa.Field, Any]]] = \
+    {TypeID.BINARY: lambda field: pa.binary(),
+     TypeID.BOOLEAN: lambda field: (pa.bool_(), False),
+     TypeID.DATE: lambda field: (pa.date32(), datetime.now()),
+     TypeID.DECIMAL: lambda field: (pa.decimal128(field.type.precision, 
field.type.scale),
+                                    decimal.Decimal()),
+     TypeID.DOUBLE: lambda field: (pa.float64(), np.nan),
+     TypeID.FIXED: lambda field: pa.binary(field.length),
+     TypeID.FLOAT: lambda field: (pa.float32(), np.nan),
+     TypeID.INTEGER: lambda field: (pa.int32(), np.nan),
+     TypeID.LIST: lambda field: (pa.list_(pa.field("element",
+                                                   
DTYPE_MAP[field.type.element_type.type_id](field.type)[0])),
+                                 None),
+     TypeID.LONG: lambda field: (pa.int64(), np.nan),
+     # To-Do: update to support reading map fields
+     # TypeID.MAP: lambda field: (,),
+     TypeID.STRING: lambda field: (pa.string(), ""),
+     TypeID.STRUCT: lambda field: (pa.struct([(nested_field.name,
+                                               
DTYPE_MAP[nested_field.type.type_id](nested_field.type)[0])
+                                              for nested_field in 
field.type.fields]), {}),
+     TypeID.TIMESTAMP: lambda field: (pa.timestamp("us"), datetime.now()),
+     # not used in SPARK, so not implementing for now
+     # TypeID.TIME: pa.time64(None)
+     }
+
+
+class ParquetReader(object):
+
+    def __init__(self, input: InputFile, expected_schema: Schema, options, 
filter_expr: Expression,
+                 case_sensitive: bool, start: int = None, end: int = None):
+        self._stats: Dict[str, int] = dict()
+
+        self._input = input
+        self._input_fo = input.new_fo()
+
+        self._arrow_file = pq.ParquetFile(self._input_fo)
+        self._file_schema = convert_parquet_to_iceberg(self._arrow_file)
+        self._expected_schema = expected_schema
+        self._file_to_expected_name_map = 
ParquetReader.get_field_map(self._file_schema,
+                                                                      
self._expected_schema)
+        self._options = options
+        self._filter = get_dataset_filter(filter_expr, 
ParquetReader.get_reverse_field_map(self._file_schema,
+                                                                               
            self._expected_schema))
+
+        self._case_sensitive = case_sensitive
+        if start is not None or end is not None:
+            raise NotImplementedError("Partial file reads are not yet 
supported")
+            # self.start = start
+            # self.end = end
+
+        self.materialized_table = False
+        self.curr_iterator = None
+        self._table = None
+        self._df = None
+
+        _logger.debug("Reader initialized for %s" % self._input.path)
+
+    @property
+    def stats(self) -> dict:
+        return dict(self._stats)
+
+    def to_pandas(self) -> Union[pd.Series, pd.DataFrame]:
+        if not self.materialized_table:
+            self._read_data()
+            with profile("to_pandas", self._stats):
+                if self._table is not None:
+                    self._df = self._table.to_pandas(use_threads=True)
+                else:
+                    self._df = None
+
+        return self._df
+
+    def to_arrow_table(self) -> pa.Table:
+        if not self.materialized_table:
+            self._read_data()
+
+        return self._table
+
+    def _read_data(self) -> None:
+        _logger.debug("Starting data read")
+
+        # only scan the columns projected and in our file
+        cols_to_read = prune_columns(self._file_schema, self._expected_schema)
+
+        with profile("read data", self._stats):
+            arrow_dataset = 
ds.FileSystemDataset.from_paths([self._input.location()],
+                                                            
schema=self._arrow_file.schema_arrow,
+                                                            
format=ds.ParquetFileFormat(),
+                                                            
filesystem=fs.LocalFileSystem())

Review comment:
       Is `LocalFileSystem` correct here? It caught my eye and wasn't sure how 
we would read eg from S3 or hadoop

##########
File path: python/iceberg/parquet/parquet_reader.py
##########
@@ -0,0 +1,240 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from datetime import datetime
+import decimal
+import logging
+from typing import Any, Callable, Dict, List, Tuple, Union
+
+from iceberg.api import Schema
+from iceberg.api.expressions import Expression
+from iceberg.api.io import InputFile
+from iceberg.api.types import NestedField, Type, TypeID
+from iceberg.core.util.profile import profile
+from iceberg.exceptions import InvalidCastException
+import numpy as np
+import pandas as pd
+import pyarrow as pa
+from pyarrow import fs
+import pyarrow.dataset as ds
+import pyarrow.parquet as pq
+
+from .dataset_utils import get_dataset_filter
+from .parquet_schema_utils import prune_columns
+from .parquet_to_iceberg import convert_parquet_to_iceberg
+
+_logger = logging.getLogger(__name__)
+
+DTYPE_MAP: Dict[TypeID,
+                Callable[[NestedField], Tuple[pa.Field, Any]]] = \
+    {TypeID.BINARY: lambda field: pa.binary(),
+     TypeID.BOOLEAN: lambda field: (pa.bool_(), False),
+     TypeID.DATE: lambda field: (pa.date32(), datetime.now()),
+     TypeID.DECIMAL: lambda field: (pa.decimal128(field.type.precision, 
field.type.scale),
+                                    decimal.Decimal()),
+     TypeID.DOUBLE: lambda field: (pa.float64(), np.nan),
+     TypeID.FIXED: lambda field: pa.binary(field.length),
+     TypeID.FLOAT: lambda field: (pa.float32(), np.nan),
+     TypeID.INTEGER: lambda field: (pa.int32(), np.nan),
+     TypeID.LIST: lambda field: (pa.list_(pa.field("element",
+                                                   
DTYPE_MAP[field.type.element_type.type_id](field.type)[0])),
+                                 None),
+     TypeID.LONG: lambda field: (pa.int64(), np.nan),
+     # To-Do: update to support reading map fields
+     # TypeID.MAP: lambda field: (,),
+     TypeID.STRING: lambda field: (pa.string(), ""),
+     TypeID.STRUCT: lambda field: (pa.struct([(nested_field.name,
+                                               
DTYPE_MAP[nested_field.type.type_id](nested_field.type)[0])
+                                              for nested_field in 
field.type.fields]), {}),
+     TypeID.TIMESTAMP: lambda field: (pa.timestamp("us"), datetime.now()),
+     # not used in SPARK, so not implementing for now
+     # TypeID.TIME: pa.time64(None)
+     }
+
+
+class ParquetReader(object):
+
+    def __init__(self, input: InputFile, expected_schema: Schema, options, 
filter_expr: Expression,
+                 case_sensitive: bool, start: int = None, end: int = None):
+        self._stats: Dict[str, int] = dict()
+
+        self._input = input
+        self._input_fo = input.new_fo()
+
+        self._arrow_file = pq.ParquetFile(self._input_fo)
+        self._file_schema = convert_parquet_to_iceberg(self._arrow_file)
+        self._expected_schema = expected_schema
+        self._file_to_expected_name_map = 
ParquetReader.get_field_map(self._file_schema,
+                                                                      
self._expected_schema)
+        self._options = options
+        self._filter = get_dataset_filter(filter_expr, 
ParquetReader.get_reverse_field_map(self._file_schema,
+                                                                               
            self._expected_schema))
+
+        self._case_sensitive = case_sensitive
+        if start is not None or end is not None:
+            raise NotImplementedError("Partial file reads are not yet 
supported")
+            # self.start = start
+            # self.end = end
+
+        self.materialized_table = False
+        self.curr_iterator = None
+        self._table = None
+        self._df = None
+
+        _logger.debug("Reader initialized for %s" % self._input.path)
+
+    @property
+    def stats(self) -> dict:
+        return dict(self._stats)
+
+    def to_pandas(self) -> Union[pd.Series, pd.DataFrame]:
+        if not self.materialized_table:
+            self._read_data()
+            with profile("to_pandas", self._stats):
+                if self._table is not None:
+                    self._df = self._table.to_pandas(use_threads=True)
+                else:
+                    self._df = None
+
+        return self._df
+
+    def to_arrow_table(self) -> pa.Table:
+        if not self.materialized_table:
+            self._read_data()
+
+        return self._table
+
+    def _read_data(self) -> None:
+        _logger.debug("Starting data read")
+
+        # only scan the columns projected and in our file
+        cols_to_read = prune_columns(self._file_schema, self._expected_schema)
+
+        with profile("read data", self._stats):
+            arrow_dataset = 
ds.FileSystemDataset.from_paths([self._input.location()],
+                                                            
schema=self._arrow_file.schema_arrow,
+                                                            
format=ds.ParquetFileFormat(),
+                                                            
filesystem=fs.LocalFileSystem())
+
+            arrow_table = arrow_dataset.to_table(columns=cols_to_read, 
filter=self._filter)
+
+        # process schema evolution if needed
+        with profile("schema_evol_proc", self._stats):
+            processed_tbl = self.migrate_schema(arrow_table)
+            for i, field in self.get_missing_fields():
+                dtype_func = DTYPE_MAP.get(field.type.type_id)
+                if dtype_func is None:
+                    raise RuntimeError("Unable to create null column for type 
%s" % field.type.type_id)
+
+                dtype = dtype_func(field)
+                processed_tbl = (processed_tbl.add_column(i,
+                                                          pa.field(field.name, 
dtype[0], True, None),
+                                                          
ParquetReader.create_null_column(processed_tbl[0],
+                                                                               
            dtype)))
+        self._table = processed_tbl
+        self.materialized_table = True
+
+    def __iter__(self):
+        return self
+
+    def __next__(self):
+        if self.curr_iterator is None:
+            if not self.materialized_table:

Review comment:
       Is there any way to do this w/o first going to pandas? Pandas is the 
most expensive part of this operation and if we could avoid it (say get tuples 
direct from pyarrow) performance would significantly improve




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to