You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/02/10 01:34:48 UTC

[GitHub] [iceberg] samredai opened a new pull request #4081: PyArrowFileIO Implementation

samredai opened a new pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081


   This PR adds the first FileIO implementation. It wraps the PyArrow [FileSystem](https://arrow.apache.org/docs/python/generated/pyarrow.fs.FileSystem.html#pyarrow.fs.FileSystem) class. The locations provided to `PyArrowInputFile` and `PyArrowOutputFile` are used to infer the correct filesystem by passing them to the [FileSystem.from_uri](https://arrow.apache.org/docs/python/generated/pyarrow.fs.FileSystem.html#pyarrow.fs.FileSystem.from_uri) method. The implementations of the `InputFile.open` and `OutputFile.create` methods both return a `pyarrow.lib.NativeFile` instance.


-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803310753



##########
File path: python/src/iceberg/io/base.py
##########
@@ -142,12 +158,25 @@ class FileIO(ABC):
 
     @abstractmethod
     def new_input(self, location: str) -> InputFile:
-        """Get an InputFile instance to read bytes from the file at the given location"""
+        """Get an InputFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): The URI to the file
+        """
 
     @abstractmethod
     def new_output(self, location: str) -> OutputFile:
-        """Get an OutputFile instance to write bytes to the file at the given location"""
+        """Get an OutputFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): The URI to the file

Review comment:
       small nit, this isn't always a URI is it?  For local file systems it is just a file path which I don't think 




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r807141137



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType

Review comment:
       Yes I would say it's a matter of not importing this module. If we import from this file anywhere else in the library then I think we'd have to do a graceful fail there (but I'm expecting that we'll just use the FileIO abc's throughout the library).




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r808583125



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem
+        filesystem = _FILESYSTEM_INSTANCES[parsed_location.scheme]
+    else:  # Instantiate a filesystem and cache it by scheme
+        filesystem, _ = FileSystem.from_uri(location)  # Infer the proper filesystem
+        _FILESYSTEM_INSTANCES[parsed_location.scheme] = filesystem  # Cache the filesystem
+    return filesystem
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyarrowFile instance
+    """
+
+    def __init__(self, location: str):
+        self._filesystem = get_filesystem(location)  # Checks for cached filesystem for this location's scheme
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self.location)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self.location)

Review comment:
       It does work for non-local file systems (at least for S3) and looks to be part of the FileSystem interface ([FileSystem.get_file_info](https://arrow.apache.org/docs/python/generated/pyarrow.fs.FileSystem.html#pyarrow.fs.FileSystem.get_file_info)). Also `type` is an attribute of the FileInfo interface ([FileInfo.type](https://arrow.apache.org/docs/python/generated/pyarrow.fs.FileInfo.html#pyarrow.fs.FileInfo.type)) that returns an enum value.
   
   Although I did find one issue here. The full uri doesn't work but the `path` value returned from `from_uri` does, which is just the path part of the URI without the scheme or authority. I'm updating this now to store `self._filesystem, self._path = FileSystem.from_uri(location)` and using `self._path` here instead of `self.location`.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810542597



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""

Review comment:
       Should this be more specific? This is actually checking whether the path exists, but it could be a directory. Maybe I'm splitting hairs 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664594



##########
File path: python/src/iceberg/io/base.py
##########
@@ -76,7 +76,15 @@ def close(self) -> None:
 
 
 class InputFile(ABC):
-    """A base class for InputFile implementations"""
+    """A base class for InputFile implementations
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for an InputFile instance
+        exists(bool): Whether the file exists or not
+    """

Review comment:
       We have `close` as part of the `InputStream` and `OutputStream` protocol. I don't think we manage the file object's state in the `InputFile` and `OutputFile` classes, do we still need a `close` there?




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810696947



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Yeah, an [OSError](https://docs.python.org/3/library/exceptions.html#OSError) just passes the [C errno](https://man7.org/linux/man-pages/man3/errno.3.html) through. Here, you'll need to check for `ENOENT`, which is no such file or directory. Same for `open_input_file`. We should catch `OSError` and look for `ENOENT` to indicate that the file doesn't exist.
   
   That probably also means that for the `open_output_stream` you can catch `OSError` and check `errno` for `EEXIST`, which indicates that the file exists.




-- 
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] samredai commented on pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#issuecomment-1034393973


   @emkornfield


-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805118117



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Good point about new schemes being supported in the future. We wouldn't have to come back here every time and update this check. The only thing is that it would be nice to fail early during an initialization of the wrapper class. Otherwise the user could do `input_file = PyArrowFile(location="foo://bar/baz.txt")` without any exception, and then later when running `input_file.open()` getting [here](https://github.com/apache/arrow/blob/master/cpp/src/arrow/filesystem/filesystem.cc#L722) and hitting `ArrowInvalid: Unrecognized filesystem type in URI: foo://bar/baz.txt`. I wonder if there's a light method available in pyarrow to just validate a URI, or maybe an array of supported schemes we could just import here and reference.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805116243



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       why is the value of scheme necessary?  For instance a new a GCS file should be present in the next version of Arrow?




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805131884



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       I haven't tried it but I assume if you pass a URI for an unsupported arrow file system to from_uri it would raise an error?  Could you do that in the constructor?   I guess the downside is it might not be cheap.  I would have to check if there is any eager work like establishing a connection when constructing a file system.  




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805189811



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       I think doing the creation on the FileIO might make sense.  Or as you note having a cache which tries to find already existing equivelant FS instantiations.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805190383



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       I sent an e-mail to dev@arrow to get more information on 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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r808653272



##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,126 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+
+def test_pyarrow_violating_input_stream_protocol():
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    input_file = PyArrowFile("foo.txt")
+    input_file._filesystem = filesystem_mock
+    with pytest.raises(TypeError) as exc_info:
+        input_file.open()
+
+    assert ("Object of type") in str(exc_info.value)
+    assert ("returned from PyArrowFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+def test_pyarrow_violating_output_stream_protocol():

Review comment:
       Thats what I was thinking.  The comment you mentioned above about having to adjust constructor above really covers my main concern in terms of consistent user experience.  It might be too heavy weight here.  I think for different implementations that access s3 (and other remote file systems) we might just need to accept that there might be quirks in behavior.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810543775



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Is it possible to pass the original exception as a cause? This assumes that any `OSError` means that the file doesn't exist, but that doesn't seem correct 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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810542435



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )

Review comment:
       When do we expect this to happen? [`open_input_file`](https://arrow.apache.org/docs/python/generated/pyarrow.fs.FileSystem.html#pyarrow.fs.FileSystem.open_input_file) states that it returns `NativeFile`, which we know is an instance of the `InputStream` protocol. I think this check probably makes sense in tests, but I would not ever expect a failure here at runtime.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810673085



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)

Review comment:
       I see. So this is always an overwrite? That seems like a problem to me, but we may not be able to do anything about it besides have the existence check above. At a minimum, we should note that this can overwrite concurrently created 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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813156611



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,188 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return file_info
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._file_info()
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists"""
+        file_info = self._file_info()
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except FileNotFoundError:
+            raise
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e):
+                raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}")
+            elif e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot open file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        try:
+            if not overwrite and self.exists() is True:
+                raise FileExistsError(f"Cannot create file, already exists: {self.location}")
+            output_file = self._filesystem.open_output_stream(self._path)
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot create file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return self
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+
+        Raises:
+            FileNotFoundError: When the file at the provided location does not exist
+            PermissionError: If the file at the provided location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e):
+                raise FileNotFoundError(f"Cannot delete file, does not exist: {location}")
+            elif e.errno == 13 or "AWS Error [code 15]" in str(e) or isinstance(e, PermissionError):

Review comment:
       @samredai, here's one that wasn't updated to a separate except clause for `PermissionError`.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803956085



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")

Review comment:
       might be worth providing more details in this error message (at least type of object?)




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805118188



##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,130 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock, patch
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowInputFile, PyArrowOutputFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowInputFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowInputFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowOutputFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowOutputFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowInputFile("foo://bar/baz.txt")
+
+    assert ("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowOutputFile("foo://bar/baz.txt")
+
+    assert ("PyArrowOutputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`") in str(exc_info.value)
+
+
+@patch("iceberg.io.pyarrow.FileSystem")
+def test_pyarrow_violating_InputStream_protocol(MockedFileSystem):
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    MockedFileSystem.from_uri.return_value = (
+        filesystem_mock,
+        "foo_path",
+    )  # Patch the FileSystem.from_uri method to return the mocked filesystem
+
+    with pytest.raises(TypeError) as exc_info:
+        PyArrowInputFile("foo.txt").open()
+
+    assert ("Object returned from PyArrowInputFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+@patch("iceberg.io.pyarrow.FileSystem")
+def test_pyarrow_violating_OutputStream_protocol(MockedFileSystem):

Review comment:
       Yeah it does look odd, I'll update 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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805116471



##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,130 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock, patch
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowInputFile, PyArrowOutputFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowInputFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowInputFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowOutputFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowOutputFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowInputFile("foo://bar/baz.txt")
+
+    assert ("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowOutputFile("foo://bar/baz.txt")
+
+    assert ("PyArrowOutputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`") in str(exc_info.value)
+
+
+@patch("iceberg.io.pyarrow.FileSystem")
+def test_pyarrow_violating_InputStream_protocol(MockedFileSystem):
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    MockedFileSystem.from_uri.return_value = (
+        filesystem_mock,
+        "foo_path",
+    )  # Patch the FileSystem.from_uri method to return the mocked filesystem
+
+    with pytest.raises(TypeError) as exc_info:
+        PyArrowInputFile("foo.txt").open()
+
+    assert ("Object returned from PyArrowInputFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+@patch("iceberg.io.pyarrow.FileSystem")
+def test_pyarrow_violating_OutputStream_protocol(MockedFileSystem):

Review comment:
       I see this pattern used elswhere but mixing camel cass and snake case doesn't look write 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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805189811



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       I think doing the creation on the FileIO might make sense.  Or as you note having a cache which tries to find already existing equivelant FS instantiations.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       I sent an e-mail to dev@arrow to get more information on 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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r808394527



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem
+        filesystem = _FILESYSTEM_INSTANCES[parsed_location.scheme]
+    else:  # Instantiate a filesystem and cache it by scheme
+        filesystem, _ = FileSystem.from_uri(location)  # Infer the proper filesystem
+        _FILESYSTEM_INSTANCES[parsed_location.scheme] = filesystem  # Cache the filesystem
+    return filesystem
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyarrowFile instance
+    """
+
+    def __init__(self, location: str):
+        self._filesystem = get_filesystem(location)  # Checks for cached filesystem for this location's scheme
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self.location)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self.location)

Review comment:
       this pattern looks strange to me, are we sure it works for non-local file systems?

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       I think maybe lets avoid the cache for now and benchmark stuff.  Based on the response I think in most cases it should hopefully be pretty cheap, but simpler is probably better to start.

##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,126 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+
+def test_pyarrow_violating_input_stream_protocol():
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    input_file = PyArrowFile("foo.txt")
+    input_file._filesystem = filesystem_mock
+    with pytest.raises(TypeError) as exc_info:
+        input_file.open()
+
+    assert ("Object of type") in str(exc_info.value)
+    assert ("returned from PyArrowFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+def test_pyarrow_violating_output_stream_protocol():

Review comment:
       it would be nice to add a test with something like minio to ensure we are getting the semantics we want on metastores.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813255178



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,190 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot get file info, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return file_info
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._file_info()
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists"""
+        file_info = self._file_info()
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except FileNotFoundError:
+            raise
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e):
+                raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}")
+            elif e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot open file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        try:
+            if not overwrite and self.exists() is True:
+                raise FileExistsError(f"Cannot create file, already exists: {self.location}")
+            output_file = self._filesystem.open_output_stream(self._path)
+        except PermissionError:

Review comment:
       This goes back to the fact that `open_output_stream` just overwrites by default. This reminded me that I was supposed to add a note describing that so I updated this docstring with:
   ```
           Note:
               This retrieves a pyarrow NativeFile by opening an output stream. If overwrite is set to False,
               a check is first performed to verify that the file does not exist. This is not thread-safe and
               a possibility does exist that the file can be created by a concurrent process after the existence
               check yet before the output stream is created. In such a case, the default pyarrow behavior will
               truncate the contents of the existing file when opening the output stream.
   ```




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810665375



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       I think this goes back to the inconsistency I described in my [other reply](https://github.com/apache/iceberg/pull/4081#discussion_r810664650). We could raise a more generic exception here and the user can see the more detailed pyarrow exception in the traceback. What do you think of something like this?:
   ```py
   from pyarrow.fs import FileSystem
   
   def delete(location):
       try:
           fs, path = FileSystem.from_uri(location)
           fs.delete_file(path)
       except Exception:
           raise RuntimeError(f"Cannot delete file: {location}")
   
   delete("s3://foo/bar.txt")
   ```
   output:
   ```py
   OSError                                   Traceback (most recent call last)
   <ipython-input-7-fd79114b19ff> in foo()
         3         fs, path = FileSystem.from_uri("s3://foo/bar.txt")
   ----> 4         fs.delete_file(path)
         5     except Exception:
   
   /usr/local/lib/python3.9/site-packages/pyarrow/_fs.pyx in pyarrow._fs.FileSystem.delete_file()
   
   /usr/local/lib/python3.9/site-packages/pyarrow/error.pxi in pyarrow.lib.check_status()
   
   OSError: When getting information for key 'bar.txt' in bucket 'foo': AWS Error [code 15]: No response body.
   
   During handling of the above exception, another exception occurred:
   
   RuntimeError                              Traceback (most recent call last)
   <ipython-input-8-c19b6d9633cf> in <module>
   ----> 1 foo()
   
   <ipython-input-7-fd79114b19ff> in foo()
         4         fs.delete_file(path)
         5     except Exception:
   ----> 6         raise RuntimeError("Cannot delete file: s3://foo/bar.txt")
         7 
         8 
   
   RuntimeError: Cannot delete file: s3://foo/bar.txt
   ```




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803879018



##########
File path: python/src/iceberg/io/base.py
##########
@@ -142,12 +158,25 @@ class FileIO(ABC):
 
     @abstractmethod
     def new_input(self, location: str) -> InputFile:
-        """Get an InputFile instance to read bytes from the file at the given location"""
+        """Get an InputFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): The URI to the file
+        """
 
     @abstractmethod
     def new_output(self, location: str) -> OutputFile:
-        """Get an OutputFile instance to write bytes to the file at the given location"""
+        """Get an OutputFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): The URI to the file

Review comment:
       Updated! If a path with no scheme is provided, it's assumed as a path to a local file. Updated the tests as well and it feels much better not having to prepend `file:` :D
   
   Also this is a decision at the implementation level so another `FileIO` implementation that does not work with local files can just raise.




-- 
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] danielcweeks commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r804871268



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):

Review comment:
       I wouldn't necessarily want to change the abstraction of InputFile/OutputFile, but I like the multiple inheritance option that @samredai suggested as long as we don't have collisions that would cause issues.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806126150



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to if a filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813110060



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except OSError as e:

Review comment:
       Rather than checking `isinstance(FileNotFoundError)` I think it would be better to throw the original. That way you don't have the longer error within an error.
   
   This should work:
   
   ```python
   try:
       ...
   except FileNotFoundError:
       raise
   except PermissionError:
       raise
   except OSError as e:
       if e.errno == 2 or "Path does not exist" in str(e):
           raise FileNotFoundError(...)
       ...
   ```




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810665375



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       I think this goes back to the inconsistency I described in my [other reply](https://github.com/apache/iceberg/pull/4081#discussion_r810664650). We could raise a more generic exception here and the user can see the more detailed pyarrow exception in the traceback:
   ```py
   from pyarrow.fs import FileSystem
   
   def delete(location):
       try:
           fs, path = FileSystem.from_uri(location)
           fs.delete_file(path)
       except Exception:
           raise RuntimeError(f"Cannot delete file: {location}")
   
   delete("s3://foo/bar.txt")
   ```
   output:
   ```py
   OSError                                   Traceback (most recent call last)
   <ipython-input-7-fd79114b19ff> in foo()
         3         fs, path = FileSystem.from_uri("s3://foo/bar.txt")
   ----> 4         fs.delete_file(path)
         5     except Exception:
   
   /usr/local/lib/python3.9/site-packages/pyarrow/_fs.pyx in pyarrow._fs.FileSystem.delete_file()
   
   /usr/local/lib/python3.9/site-packages/pyarrow/error.pxi in pyarrow.lib.check_status()
   
   OSError: When getting information for key 'bar.txt' in bucket 'foo': AWS Error [code 15]: No response body.
   
   During handling of the above exception, another exception occurred:
   
   RuntimeError                              Traceback (most recent call last)
   <ipython-input-8-c19b6d9633cf> in <module>
   ----> 1 foo()
   
   <ipython-input-7-fd79114b19ff> in foo()
         4         fs.delete_file(path)
         5     except Exception:
   ----> 6         raise RuntimeError("Cannot delete file: s3://foo/bar.txt")
         7 
         8 
   
   RuntimeError: Cannot delete file: s3://foo/bar.txt
   ```




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664660



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)

Review comment:
       It definitely can, [updated](https://github.com/apache/iceberg/pull/4081/commits/f960ec9ee28bb50c5cf3fe68d20315f8682c3a33)!




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664635



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"

Review comment:
       [Updated](https://github.com/apache/iceberg/pull/4081/commits/bd0f238a6f84a54ee86b01801b8b38a0b885e667)! Thanks, I'll make a note to keep the exception messages consistent going forward.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664650



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)

Review comment:
       Ah I should spell that out in the docstring. The throw is not consistent here in that if it's a local file that doesn't exist, pyarrow throws `FileNotFoundError` however if it's an S3 path it throws `OSError`. We can do a generic `Exception` catch and consistently raise a `FileNotFoundError` with the raw pyarrow error in the traceback. Or should we just defer to whatever exception pyarrow raises?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664603



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""

Review comment:
       [Updated](https://github.com/apache/iceberg/pull/4081/commits/bd6ed38523e25f13a80e0c6276834412ed9fa9dc) this to say "Checks whether the location exists" which makes more sense generally because the attribute is named `location`.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805118540



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):

Review comment:
       I consolidated `PyArrowInputFile` and `PyArrowOutputFile` into a single `PyArrowFile` class that inherits from both the `InputFile` and `OutputFile` base classes.




-- 
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] samredai commented on pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#issuecomment-1043763087


   @rdblue I believe I've addressed all of the comments here and it should be ready for a final review. Once we get to parts of the code that are using FileIO instances I think some aggressive testing there should suss out any bad assumptions 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r804031877



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:

Review comment:
       Do you mean that we can use `-> InputStream` here? I thought the protocols would be used just to typehint the abstract classes. This method would always return a `NativeFile` though based on the code, so wouldn't it be a more informative typehint to specify this?
   
   On the other hand, we do want to bind usage of this return object in the rest of the library to those methods defined in the protocol, so thinking about this some more feels like specifying the protocol as the return type would be better (much better in fact). I'll update this but let me know if I'm understanding this correctly.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806143570



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem
+        filesystem = _FILESYSTEM_INSTANCES[parsed_location.scheme]
+    else:  # Instantiate a filesystem and cache it by scheme
+        filesystem, _ = FileSystem.from_uri(location)  # Infer the proper filesystem
+        _FILESYSTEM_INSTANCES[parsed_location.scheme] = filesystem  # Cache the filesystem

Review comment:
       I think we'll want to use WeakRef 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806229890



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       Hmm, I see, I just read Weston's response. I'll revert this commit and just do `from_uri` in the `PyArrowFile` constructor. Maybe we can then just wait to optimize that and think through some approaches at that time. How does that sound?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664650



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)

Review comment:
       Ah I should spell that out in the docstring. The throw is not consistent here in that if it's a local file that doesn't exist, pyarrow throws `FileNotFoundError` however if it's an S3 path it throws `OSError`. We can do a generic `Exception` catch and consistently raise a `FileNotFoundError` with the raw pyarrow error in the traceback. Or should we just defer to whatever exception pyarrow raises and pass through that inconsistency?

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)

Review comment:
       Ah I should spell that out in the docstring. The throw is not consistent here in that if it's a local file that doesn't exist, pyarrow throws `FileNotFoundError` however if it's an S3 path it throws `OSError`. We can do a generic `Exception` catch and consistently raise a `FileNotFoundError` with the raw pyarrow error in the traceback. Or should we just defer to whatever exception pyarrow raises and pass through that inconsistency to the user?




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810542931



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"

Review comment:
       In Iceberg, we try to stick to brief and direct error messages. Those typically look like this:
   * `Cannot <what you tried>: <reason with context> (<optional help>)`
   * `Failed to <low-level operation>: <reason with context>` (and wrap the underlying exception if there is one)
   * `Invalid <something>: <what was invalid>`
   
   Here, I'd recommend `"Cannot create file, already exists: {self.location}". I think it's fairly clear that you can pass an overwrite flag so I wouldn't mention it necessarily.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"

Review comment:
       In Iceberg, we try to stick to brief and direct error messages. Those typically look like this:
   * `Cannot <what you tried>: <reason with context> (<optional help>)`
   * `Failed to <low-level operation>: <reason with context>` (and wrap the underlying exception if there is one)
   * `Invalid <something>: <what was invalid>`
   
   Here, I'd recommend `"Cannot create file, already exists: {self.location}"`. I think it's fairly clear that you can pass an overwrite flag so I wouldn't mention it necessarily.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810543749



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)

Review comment:
       Can't this return `self`? I don't think that this class carries any state that needs to be reset.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664561



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property

Review comment:
       Good point, I switched this to a method here and also in the `InputFile` and `OutputFile` base classes.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)

Review comment:
       Definitely better to use `get_file_info`, updated!

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )

Review comment:
       You're right I think it's good enough to just include this in tests. I was thinking we would catch a change in the behavior of what pyarrow returns from `open_input_file`, but that would be something we catch in tests after updating the version or something along those lines, not at runtime.

##########
File path: python/src/iceberg/io/base.py
##########
@@ -76,7 +76,15 @@ def close(self) -> None:
 
 
 class InputFile(ABC):
-    """A base class for InputFile implementations"""
+    """A base class for InputFile implementations
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for an InputFile instance
+        exists(bool): Whether the file exists or not
+    """

Review comment:
       We have `close` as part of the `InputStream` and `OutputStream` protocol. I don't think we manage the file object's state in the `InputFile` and `OutputFile` classes, do we still need a `close` method in that case?

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""

Review comment:
       Updated this to say "Checks whether the location exists" which makes more sense generally because the attribute is named `location`.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"

Review comment:
       Updated! Thanks, I'll make a note to keep the exception messages consistent going forward.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)

Review comment:
       Did you mean if opening the file succeeds? For `open_output_stream`, the docstring says `If the target already exists, existing data is truncated.`. That would be a problem here where if `self.exists()` on the line above returns `False`, then a concurrent process creates and writes the files, then the `open_output_stream` here would successfully open the output stream but would truncate the contents. So it would effectively overwrite it even though `overwrite=False`.
   
   @emkornfield, is there a way with pyarrow to open an output stream in a way that fails if the file already exists?

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)

Review comment:
       Ah I should spell that out in the docstring. The throw is not consistent here in that if it's a local file that doesn't exist, pyarrow throws a `FileNotFoundError` however if it's an S3 path it raises `OSError`. We can do a generic `Exception` catch and consistently raise a `FileNotFoundError` with the raw pyarrow error in the traceback. Or should we just defer to whatever exception pyarrow raises?




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813099151



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error

Review comment:
       I would probably move this to `_file_info()` method that handles the exception and is shared by `__len__` and `exists`. Otherwise, you don't handle the exception in the `__len__` call to `get_file_info`.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813170822



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,190 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot get file info, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return file_info
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._file_info()
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists"""
+        file_info = self._file_info()
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except FileNotFoundError:
+            raise
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e):
+                raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}")
+            elif e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot open file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        try:
+            if not overwrite and self.exists() is True:
+                raise FileExistsError(f"Cannot create file, already exists: {self.location}")
+            output_file = self._filesystem.open_output_stream(self._path)
+        except PermissionError:

Review comment:
       Do we need to catch and raise FileExistsError?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r808648464



##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,126 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+
+def test_pyarrow_violating_input_stream_protocol():
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    input_file = PyArrowFile("foo.txt")
+    input_file._filesystem = filesystem_mock
+    with pytest.raises(TypeError) as exc_info:
+        input_file.open()
+
+    assert ("Object of type") in str(exc_info.value)
+    assert ("returned from PyArrowFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+def test_pyarrow_violating_output_stream_protocol():

Review comment:
       Do you mean like adding a test suite that launches a local minio server and uses it to mock s3?




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810542114



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)

Review comment:
       It looks like the [size is available from `get_file_info`](https://arrow.apache.org/docs/python/generated/pyarrow.fs.FileInfo.html#pyarrow.fs.FileInfo). Is it better to open the file to get the size? I would expect that `get_file_info` is better because it doesn't create an open file handle that needs to be closed.
   
   If this is better for some reason, I'm fine using this. But we do need to close it if we don't switch to `get_file_info`.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810543670



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)

Review comment:
       What is thrown when `open_input_file` is called for a file that doesn't exist?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664588



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )

Review comment:
       You're right I think it's good enough to just include this in tests. I was thinking we would catch a change in the behavior of what pyarrow returns from `open_input_file`, but that would be something we catch in tests after updating the version or something along those lines, not at runtime.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)

Review comment:
       Definitely better to use `get_file_info`, [updated](https://github.com/apache/iceberg/pull/4081/commits/8da75463bca649223cbf935f00712ab51a634700)!




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806126150



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to determine if a cached filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806126150



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to determine if a cached filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`. The constructor for `PyArrowFile` then just does `self._filesystem = get_filesystem(location)` and uses that attribute throughout the other 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806126150



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to determine if a cached filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`. The constructor for `PyArrowFile` then just does `self._filesystem = get_filesystem(location)`.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806143570



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem
+        filesystem = _FILESYSTEM_INSTANCES[parsed_location.scheme]
+    else:  # Instantiate a filesystem and cache it by scheme
+        filesystem, _ = FileSystem.from_uri(location)  # Infer the proper filesystem
+        _FILESYSTEM_INSTANCES[parsed_location.scheme] = filesystem  # Cache the filesystem

Review comment:
       I think we'll want to use WeakRef here?  

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       I don't think Scheme is enough if we want this to be effective.  I believe 's3://` the first path component is bucket which implies different connections.  Based on Weston's response we would either want to eat the cost of create the file system each time or at least specialized for S3 to cache based on bucket.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810673142



##########
File path: python/src/iceberg/io/base.py
##########
@@ -76,7 +76,15 @@ def close(self) -> None:
 
 
 class InputFile(ABC):
-    """A base class for InputFile implementations"""
+    """A base class for InputFile implementations
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for an InputFile instance
+        exists(bool): Whether the file exists or not
+    """

Review comment:
       I must have missed it. I didn't see it on the protocol, but that's where I think it should be.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810543775



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Is it possible to pass the original exception as a cause? This assumes that any `OSError` means that the file doesn't exist, but that doesn't seem correct to me.
   
   Also, I'd recommend the error message: `"Cannot delete file, does not exist: {str_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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810696947



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Yeah, an [OSError](https://docs.python.org/3/library/exceptions.html#OSError) just passes the [C errno](https://man7.org/linux/man-pages/man3/errno.3.html) through. Here, you'll need to check for `ENOENT`, which is no such file or directory.
   
   That probably also means that for the `open_output_stream` you can catch `OSError` and check `errno` for `EEXIST`, which indicates that the file exists.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r809176195



##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,126 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowFile("foo://bar/baz.txt")
+
+    assert ("Unrecognized filesystem type in URI") in str(exc_info.value)
+
+
+def test_pyarrow_violating_input_stream_protocol():
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    input_file = PyArrowFile("foo.txt")
+    input_file._filesystem = filesystem_mock
+    with pytest.raises(TypeError) as exc_info:
+        input_file.open()
+
+    assert ("Object of type") in str(exc_info.value)
+    assert ("returned from PyArrowFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+def test_pyarrow_violating_output_stream_protocol():

Review comment:
       I think that's a great idea but maybe something bigger to tackle in a follow-up PR. I definitely agree that we need to have some strong integration tests for different storage providers that test for consistency across different FileIO implementations. 👍




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664603



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""

Review comment:
       [Updated](https://github.com/apache/iceberg/pull/4081/commits/8198bfd8d791957b8858cc838d73e90b38ffc58e) this to say "Checks whether the location exists" which makes more sense generally because the attribute is named `location`.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810734134



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Digging some more I've found some more diverging behavior. For local files, `e.errno` is 2 as expected to represent `ENOENT` when the file does not exist.
   
   For S3 though, there's no value set at all for `e.errno`. Instead:
   
   - If the issue is access denied to the bucket, it hits this [ErrorToStatus](https://github.com/apache/arrow/blob/master/cpp/src/arrow/filesystem/s3_internal.h#L112) function that returns an `OSError` with nothing but a string description containing "AWS Error [code 15]" which corresponds to [ACCESS_DENIED](https://sdk.amazonaws.com/cpp/api/LATEST/_s3_errors_8h_source.html) in the aws cpp sdk.
   - If the bucket access is fine but the resource doesn't exist, it hits this [PathNotFound](https://github.com/apache/arrow/blob/master/cpp/src/arrow/filesystem/s3fs.cc#L2143) error which returns an `OSError` with nothing but a string description containing "Path does not exist: foo".
   
   I'm not enthusiastic about error handling based on substring checks, but the only thing I can think of here is checking for either `e.errno=2` or an error message containing "Path does not exist" in order to appropriately raise a `FileNotFoundError`. That feels fragile but should handle local and s3 files. I haven't checked the behavior for HDFS files but that could also be 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] rdblue commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813170528



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,190 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)

Review comment:
       Catch and raise FileNotFoundException?




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803955842



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):

Review comment:
       For implementations, is there any reason to have 2 completely seperate classes?  Can't we have one that implements both?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803831390



##########
File path: python/src/iceberg/io/base.py
##########
@@ -142,12 +158,25 @@ class FileIO(ABC):
 
     @abstractmethod
     def new_input(self, location: str) -> InputFile:
-        """Get an InputFile instance to read bytes from the file at the given location"""
+        """Get an InputFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): The URI to the file
+        """
 
     @abstractmethod
     def new_output(self, location: str) -> OutputFile:
-        """Get an OutputFile instance to write bytes to the file at the given location"""
+        """Get an OutputFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): The URI to the file

Review comment:
       As it stands the user would have to use `file://...` but I totally agree. The code here should make an assumption that if the URI is missing a scheme that it's just a local filepath. I'll update the code and change the docstring to say "A URI or a path to a local 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r804020699



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):

Review comment:
       Hmm, I was initially just following the breakdown of the abstract classes but I suppose you're right. This could be `class PyArrowFile(InputFile, OutputFile):`. @danielcweeks or @rdblue, are there any unforeseen implications of doing this here? We could still have `PyArrowFileIO.new_input` and `PyArrowFileIO.new_output` methods but they would both basically return the same thing, a `PyArrowFile` instance. The instance in both cases would be both readable and writable 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805137594



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Yes from_uri seems to catch it. I'm thinking a bit more about this and I feel like it may also be worth avoiding the cost of instantiating a FileSystem for each call to the "open" or "create" method. Maybe we can use some kind of singleton wrapper that re-uses FileSystem instances (@CircArgs curious about your thoughts here). Is there any downside to keeping a FileSystem instance around for longer?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805115424



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):
+    """An OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for writing
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowOutputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowOutputFile
+        >>> output_file = PyArrowOutputFile("s3://foo/bar.txt")
+        >>> output_file.create().write(b'baz')
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",

Review comment:
       Great idea, I'll add a `SUPPORTED_SCHEMES` variable at the module level.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803956584



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:

Review comment:
       Is there a reason to break abstraction here of the protocols 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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r804880160



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:

Review comment:
       Yes, I would thinking of `-> InputStream`, I haven't coded much in python since typehints have come about, but it feels like generally for method overrides its not clear if you want specify a more specific type then the method that is getting overridden suggests.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805878308



##########
File path: python/tests/io/test_pyarrow.py
##########
@@ -0,0 +1,130 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import tempfile
+from unittest.mock import MagicMock, patch
+
+import pytest
+from pyarrow.fs import FileType
+
+from iceberg.io.pyarrow import PyArrowInputFile, PyArrowOutputFile
+
+
+def test_pyarrow_input_file():
+    """Test reading a file using PyArrowInputFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+        with open(file_location, "wb") as f:
+            f.write(b"foo")
+
+        # Confirm that the file initially exists
+        assert os.path.exists(file_location)
+
+        # Instantiate the input file
+        absolute_file_location = os.path.abspath(file_location)
+        input_file = PyArrowInputFile(location=f"{absolute_file_location}")
+
+        # Test opening and reading the file
+        f = input_file.open()
+        data = f.read()
+        assert data == b"foo"
+        assert len(input_file) == 3
+
+
+def test_pyarrow_output_file():
+    """Test writing a file using PyArrowOutputFile"""
+
+    with tempfile.TemporaryDirectory() as tmpdirname:
+        file_location = os.path.join(tmpdirname, "foo.txt")
+
+        # Instantiate the output file
+        absolute_file_location = os.path.abspath(file_location)
+        output_file = PyArrowOutputFile(location=f"{absolute_file_location}")
+
+        # Create the output file and write to it
+        f = output_file.create()
+        f.write(b"foo")
+
+        # Confirm that bytes were written
+        with open(file_location, "rb") as f:
+            assert f.read() == b"foo"
+
+        assert len(output_file) == 3
+
+
+def test_pyarrow_invalid_scheme():
+    """Test that a ValueError is raised if a location is provided with an invalid scheme"""
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowInputFile("foo://bar/baz.txt")
+
+    assert ("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`") in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        PyArrowOutputFile("foo://bar/baz.txt")
+
+    assert ("PyArrowOutputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`") in str(exc_info.value)
+
+
+@patch("iceberg.io.pyarrow.FileSystem")
+def test_pyarrow_violating_InputStream_protocol(MockedFileSystem):
+    """Test that a TypeError is raised if an input file is provided that violates the InputStream protocol"""
+
+    # Missing seek, tell, closed, and close
+    input_file_mock = MagicMock(spec=["read"])
+
+    # Create a mocked filesystem that returns input_file_mock
+    filesystem_mock = MagicMock()
+    filesystem_mock.open_input_file.return_value = input_file_mock
+
+    MockedFileSystem.from_uri.return_value = (
+        filesystem_mock,
+        "foo_path",
+    )  # Patch the FileSystem.from_uri method to return the mocked filesystem
+
+    with pytest.raises(TypeError) as exc_info:
+        PyArrowInputFile("foo.txt").open()
+
+    assert ("Object returned from PyArrowInputFile.open does not match the InputStream protocol.") in str(exc_info.value)
+
+
+@patch("iceberg.io.pyarrow.FileSystem")
+def test_pyarrow_violating_OutputStream_protocol(MockedFileSystem):

Review comment:
       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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r807101201



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType

Review comment:
       Can we gracefully fail if pyarrow is not present? Or is that a matter of not importing this module?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r807015392



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       I just realized urlparse actually parses the bucket name as `netloc` which makes sense since the bucket name is the authority in the URI. For example:
   ```py
   from urllib.parse import urlparse
   urlparse("s3://foobucket/test.txt")
   # ParseResult(scheme='s3', netloc='foobucket', path='/test.txt', params='', query='', fragment='')
   ```
   So we could cache for S3 only based on bucket like you suggested by using a tuple as the key. Something like this:
   ```py
   parsed_location = urlparse(location)
   if parsed_location.scheme == "s3":
     s3_fs_lookup = (parsed_location.scheme, parsed_location.netloc)
     if _FILESYSTEM_INSTANCES.get(s3_fs_lookup):
       filesystem = _FILESYSTEM_INSTANCES[s3_fs_lookup]
     else:
       filesystem, _ = FileSystem.from_uri(location)
       _FILESYSTEM_INSTANCES[s3_fs_lookup] = filesystem
       return filesystem
   
   filesystem, _ = FileSystem.from_uri(location)
   return filesystem
   ```




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810541940



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property

Review comment:
       Why is this a property instead of a method? Since this will commonly make an RPC call and isn't cached, it seems like something we'd want people to think of as a method 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] rdblue commented on pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#issuecomment-1049184602


   Nice work, @samredai!


-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813116568



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)

Review comment:
       Yes helping with debugging was exactly what I was thinking. It's possible but harder to set a `pdb.set_trace()` in the right spot when there's a method call as part of the return statement like `return self._filesystem.open_input_file(self._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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813108352



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e) or isinstance(e, FileNotFoundError):
+                raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}")

Review comment:
       Today I learned that Python automatically tracks causes from exceptions raised within except blocks. That's great; one of the annoying things about Java is that people forget to include the cause when they throw a new 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


[GitHub] [iceberg] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813253544



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,190 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)

Review comment:
       If the file does not exist pyarrow actually does not raise. It returns a FileInfo object with a type attribute of `FileType.NotFound`. So I added a check for that here and raise. 👍 
   
   I then changed the `exists()` method to just do a try except where it catches the `FileNotFoundError` from `_file_info()`.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813137673



##########
File path: python/src/iceberg/io/base.py
##########
@@ -76,7 +76,15 @@ def close(self) -> None:
 
 
 class InputFile(ABC):
-    """A base class for InputFile implementations"""
+    """A base class for InputFile implementations
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for an InputFile instance
+        exists(bool): Whether the file exists or not

Review comment:
       Nope, I removed it here. I also removed filesystem since that's only used internally and not even set as a property.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r803879018



##########
File path: python/src/iceberg/io/base.py
##########
@@ -142,12 +158,25 @@ class FileIO(ABC):
 
     @abstractmethod
     def new_input(self, location: str) -> InputFile:
-        """Get an InputFile instance to read bytes from the file at the given location"""
+        """Get an InputFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): The URI to the file
+        """
 
     @abstractmethod
     def new_output(self, location: str) -> OutputFile:
-        """Get an OutputFile instance to write bytes to the file at the given location"""
+        """Get an OutputFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): The URI to the file

Review comment:
       Updated! If a path with no scheme is provided, it's assumed as a path to a local file. Updated the tests as well and it feels much better not having to prepend `file:` :D




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810542114



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)

Review comment:
       It looks like the size is available from `get_file_info`. Is it better to open the file to get the size? I would expect that `get_file_info` is better because it doesn't create an open file handle that needs to be closed.
   
   If this is better for some reason, I'm fine using this. But we do need to close it if we don't switch to `get_file_info`.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810673873



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Rather than RuntimeError, I'd use a more specific error type, but I like the try/except block to normalize this.
   
   For OSError specifically, I think we also need to see what kind of OSError it is and allow it through if it isn't the one that means the file is missing.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810672939



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)

Review comment:
       I think we should make the errors consistent for the caller and should document them in the Protocol.




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813095068



##########
File path: python/src/iceberg/io/base.py
##########
@@ -76,7 +76,15 @@ def close(self) -> None:
 
 
 class InputFile(ABC):
-    """A base class for InputFile implementations"""
+    """A base class for InputFile implementations
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for an InputFile instance
+        exists(bool): Whether the file exists or not

Review comment:
       This is no longer an attribute/property 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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813110683



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)

Review comment:
       Can you return from here rather than using `input_file`? Or is this assignment to help with debugging?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813136987



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except OSError as e:

Review comment:
       Updated!

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,179 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error

Review comment:
       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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813157239



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,188 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return file_info
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._file_info()
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists"""
+        file_info = self._file_info()
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except FileNotFoundError:
+            raise
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e):
+                raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}")
+            elif e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot open file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        try:
+            if not overwrite and self.exists() is True:
+                raise FileExistsError(f"Cannot create file, already exists: {self.location}")
+            output_file = self._filesystem.open_output_stream(self._path)
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot create file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return self
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+
+        Raises:
+            FileNotFoundError: When the file at the provided location does not exist
+            PermissionError: If the file at the provided location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError as e:

Review comment:
       Should this catch and raise FileNotFoundException to pass it through as well?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r813257811



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,188 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileInfo, FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def _file_info(self) -> FileInfo:
+        """Retrieves a pyarrow.fs.FileInfo object for the location
+
+        Raises:
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            file_info = self._filesystem.get_file_info(self._path)
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot check if file exists, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return file_info
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file_info = self._file_info()
+        return file_info.size
+
+    def exists(self) -> bool:
+        """Checks whether the location exists"""
+        file_info = self._file_info()
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+
+        Raises:
+            FileNotFoundError: If the file at self.location does not exist
+            PermissionError: If the file at self.location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        try:
+            input_file = self._filesystem.open_input_file(self._path)
+        except FileNotFoundError:
+            raise
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 2 or "Path does not exist" in str(e):
+                raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}")
+            elif e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot open file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        try:
+            if not overwrite and self.exists() is True:
+                raise FileExistsError(f"Cannot create file, already exists: {self.location}")
+            output_file = self._filesystem.open_output_stream(self._path)
+        except PermissionError:
+            raise
+        except OSError as e:
+            if e.errno == 13 or "AWS Error [code 15]" in str(e):
+                raise PermissionError(f"Cannot create file, access denied: {self.location}")
+            raise  # pragma: no cover - If some other kind of OSError, raise the raw error
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return self
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+
+        Raises:
+            FileNotFoundError: When the file at the provided location does not exist
+            PermissionError: If the file at the provided location cannot be accessed due to a permission error such as
+                an AWS error code 15
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError as e:

Review comment:
       Yes it should since that's raised for local files, 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805115344



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:

Review comment:
       I just did a quick test and mypy will indeed check the overrides against the protocol. This is the right flexibility since if for example pyarrow one day decided to return `CustomFooFile` instead of `NativeFile`, mypy would raise the following **_only_** if it's missing the methods defined in the protocol:
   ```
   Return type "CustomFooFile" of "open" incompatible with return type "InputStream" in supertype "InputFile"
   ```
   So it feels like what we want--to only enforce adherence to the protocols.




-- 
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 merged pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081


   


-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r811555297



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)
+
+
+class PyArrowFileIO(FileIO):
+    def new_input(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to read bytes from the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def new_output(self, location: str) -> PyArrowFile:
+        """Get a PyArrowFile instance to write bytes to the file at the given location
+
+        Args:
+            location(str): A URI or a path to a local file
+
+        Returns:
+            PyArrowFile: A PyArrowFile instance for the given location
+        """
+        return PyArrowFile(location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
+        """Delete the file at the given location
+
+        Args:
+            location(str, InputFile, OutputFile): The URI to the file--if an InputFile instance or an
+            OutputFile instance is provided, the location attribute for that instance is used as the location
+            to delete
+        """
+        str_path = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        filesystem, path = FileSystem.from_uri(str_path)  # Infer the proper filesystem
+        try:
+            filesystem.delete_file(path)
+        except OSError:
+            raise FileNotFoundError(f"File could not be deleted because it does not exist: {str_path}")

Review comment:
       Updated the PR with this approach. The `exists`, `open`, and `create` methods now catch and appropriately raise errors such as `PermissionError`, `FileExistsError`, or `FileNotFoundError`.
   
   For local files this is straightforward and checks `e.errno`, but for S3 errors, it checks the string response. If the error raised can't be determined, the underlying pyarrow error is just raised directly. Even when it's caught, the underlying pyarrow error is always available in the traceback.
   
   I've also added a handful of tests to test for each of these scenarios and added docs for all of them in the `Raises:` section of the method docstrings.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r811556499



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )

Review comment:
       Fixed in [this](https://github.com/apache/iceberg/pull/4081/commits/7fa28c766c3dd5bd6702abcf4d85860c6bbb164c) commit.




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664588



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )

Review comment:
       You're right I think it's good enough to just include this in tests. I was thinking we would catch a change in the behavior of what pyarrow returns from `open_input_file`, but that would be something we catch in tests after updating the version or something along those lines, not at runtime. ([update commit](https://github.com/apache/iceberg/pull/4081/commits/8da75463bca649223cbf935f00712ab51a634700))




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664561



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property

Review comment:
       Good point, I switched this to a method here and also in the `InputFile` and `OutputFile` base classes. ([update commit](https://github.com/apache/iceberg/pull/4081/commits/8198bfd8d791957b8858cc838d73e90b38ffc58e))




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810543616



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)

Review comment:
       If opening fails because the file was created concurrently, does it arrow raise `FileExistsError`?




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810542515



##########
File path: python/src/iceberg/io/base.py
##########
@@ -76,7 +76,15 @@ def close(self) -> None:
 
 
 class InputFile(ABC):
-    """A base class for InputFile implementations"""
+    """A base class for InputFile implementations
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for an InputFile instance
+        exists(bool): Whether the file exists or not
+    """

Review comment:
       I just realized that we're missing a method for both `InputFile` and `OutputFile`: `close`! It's not needed to read or write, but it's really important!




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r810664660



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,154 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowFile(InputFile, OutputFile):
+    """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowFile instance
+        exists(bool): Whether the file exists or not
+        filesystem(pyarrow.fs.FileSystem): An implementation of the FileSystem base class inferred from the location
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowFile
+        >>> input_file = PyArrowFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()  # Read the contents of the PyArrowFile instance
+        >>> output_file = PyArrowFile("s3://baz/qux.txt")
+        >>> output_file.create().write(b'foobytes')  # Write bytes to the PyArrowFile instance
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+            self._filesystem, self._path = FileSystem.from_uri(os.path.abspath(location))
+        else:
+            self._filesystem, self._path = FileSystem.from_uri(location)  # Infer the proper filesystem
+        super().__init__(location=location)
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        file = self._filesystem.open_input_file(self._path)
+        return file.size()
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        file_info = self._filesystem.get_file_info(self._path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> InputStream:
+        """Opens the location using a PyArrow FileSystem inferred from the location
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`
+        """
+        input_file = self._filesystem.open_input_file(self._path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError(
+                f"Object of type {type(input_file)} returned from PyArrowFile.open does not match the InputStream protocol."
+            )
+        return input_file
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location
+
+        Args:
+            overwrite(bool): Whether to overwrite the file if it already exists
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+
+        Raises:
+            FileExistsError: If the file already exists at `self.location` and `overwrite` is False
+        """
+        if not overwrite and self.exists:
+            raise FileExistsError(
+                f"A file already exists at this location. If you would like to overwrite it, set `overwrite=True`: {self.location}"
+            )
+        output_file = self._filesystem.open_output_stream(self._path)
+        if not isinstance(output_file, OutputStream):
+            raise TypeError(
+                f"Object of type {type(output_file)} returned from PyArrowFile.create(...) does not match the OutputStream protocol."
+            )
+        return output_file
+
+    def to_input_file(self) -> "PyArrowFile":
+        """Returns a new PyArrowFile for the location of an existing PyArrowFile instance
+
+        This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
+        PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
+        a copy of the same instance.
+        """
+        return PyArrowFile(self.location)

Review comment:
       It definitely can, 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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806126150



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to if a filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to determine if a cached filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to determine if a cached filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`. The constructor for `PyArrowFile` then just does `self._filesystem = get_filesystem(location)`.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")

Review comment:
       Based on the response on dev@arrow, it looks like caching here is reasonable. I updated the PR to do this and it actually streamlined a few things (removed the need for `parsed_location` to be a class attribute and made some patching in tests a bit easier where you can just override self._filesystem now).
   
   The approach I took was adding a module level `_FILESYSTEM_INSTANCES` dictionary to use as a cache and a module level function `get_filesystem`. The function takes a location, pulls out the scheme, and then checks `_FILESYSTEM_INSTANCES` to determine if a cached filesystem already exists for that scheme. If the key doesn't exist, it instantiates a new filesystem using `FileSystem.from_uri(location)` and adds that filesystem instance to `_FILESYSTEM_INSTANCES`. The constructor for `PyArrowFile` then just does `self._filesystem = get_filesystem(location)` and uses that attribute throughout the other methods.

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       Hmm, I see, I just read Weston's response. I'll revert this commit and just do `from_uri` in the `PyArrowFile` constructor. Maybe we can then just wait to optimize that and think through some approaches at that time. How does that sound?

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       I just realized urlparse actually parses the bucket name as `netloc` which makes sense since the bucket name is the authority in the URI. For example:
   ```py
   from urllib.parse import urlparse
   urlparse("s3://foobucket/test.txt")
   # ParseResult(scheme='s3', netloc='foobucket', path='/test.txt', params='', query='', fragment='')
   ```
   So we could cache for S3 only based on bucket like you suggested by using a tuple as the key. Something like this:
   ```py
   parsed_location = urlparse(location)
   if parsed_location.scheme == "s3":
     s3_fs_lookup = (parsed_location.scheme, parsed_location.netloc)
     if _FILESYSTEM_INSTANCES.get(s3_fs_lookup):
       filesystem = _FILESYSTEM_INSTANCES[s3_fs_lookup]
     else:
       filesystem, _ = FileSystem.from_uri(location)
       _FILESYSTEM_INSTANCES[s3_fs_lookup] = filesystem
       return filesystem
   
   filesystem, _ = FileSystem.from_uri(location)
   return filesystem
   ```

##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType

Review comment:
       Yes I would say it's a matter of not importing this module. If we import from this file anywhere else in the library then I think we'd have to do a graceful fail there (but I'm expecting that we'll just use the FileIO abc's throughout the library).




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805118438



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")

Review comment:
       Added the type of the object to the error message. 👍 




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805118239



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,213 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): The URI to the file
+
+    Attributes:
+        location(str): The URI to the file
+        parsed_location(urllib.parse.ParseResult): The parsed location URI
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:

Review comment:
       Updated the return types to be `InputStream` and `OutputStream`. 👍 




-- 
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 change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r807101201



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType

Review comment:
       Can we gracefully fail if pyarrow is not present? Or is that a matter of not importing this module?




-- 
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] samredai commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805117419



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):
+    """An OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for writing
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowOutputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowOutputFile
+        >>> output_file = PyArrowOutputFile("s3://foo/bar.txt")
+        >>> output_file.create().write(b'baz')
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",

Review comment:
       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] kbendick commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r805075063



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,215 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesytem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+from typing import Union
+from urllib.parse import ParseResult, urlparse
+
+from pyarrow import NativeFile
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+
+class PyArrowInputFile(InputFile):
+    """An InputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for reading
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowInputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowInputFile
+        >>> input_file = PyArrowInputFile("s3://foo/bar.txt")
+        >>> file_content = input_file.open().read()
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",
+        ):  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("PyArrowInputFile location must have a scheme of `file`, `mock`, `s3fs`, `hdfs`, or `viewfs`")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    def __len__(self) -> int:
+        """Returns the total length of the file, in bytes"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file = filesytem.open_input_file(path)
+        return file.size()
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    @property
+    def exists(self) -> bool:
+        """Checks whether the file exists"""
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        file_info = filesytem.get_file_info(path)
+        return False if file_info.type == FileType.NotFound else True
+
+    def open(self) -> NativeFile:
+        """Opens the location using a PyArrow FileSystem inferred from the scheme
+
+        Returns:
+            pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location
+        """
+        filesytem, path = FileSystem.from_uri(self.location)  # Infer the proper filesystem
+        input_file = filesytem.open_input_file(path)
+        if not isinstance(input_file, InputStream):
+            raise TypeError("""Object returned from PyArrowInputFile.open does not match the InputStream protocol.""")
+        return input_file
+
+
+class PyArrowOutputFile(OutputFile):
+    """An OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances for writing
+
+    Args:
+        location(str): A URI or a path to a local file
+
+    Attributes:
+        location(str): The URI or path to a local file for a PyArrowOutputFile instance
+        parsed_location(urllib.parse.ParseResult): The parsed location with attributes `scheme`, `netloc`, `path`, `params`,
+          `query`, and `fragment`
+        exists(bool): Whether the file exists or not
+
+    Examples:
+        >>> from iceberg.io.pyarrow import PyArrowOutputFile
+        >>> output_file = PyArrowOutputFile("s3://foo/bar.txt")
+        >>> output_file.create().write(b'baz')
+    """
+
+    def __init__(self, location: str):
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+
+        if parsed_location.scheme and parsed_location.scheme not in (
+            "file",
+            "mock",
+            "s3fs",
+            "hdfs",
+            "viewfs",

Review comment:
       Nit: These are repeated multiple times. Could we just add them to a list or something somewhere and then reference that? The list could even be used in the raised exception too.




-- 
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] emkornfield commented on a change in pull request #4081: PyArrowFileIO Implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #4081:
URL: https://github.com/apache/iceberg/pull/4081#discussion_r806144601



##########
File path: python/src/iceberg/io/pyarrow.py
##########
@@ -0,0 +1,178 @@
+# 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.
+"""FileIO implementation for reading and writing table files that uses pyarrow.fs
+
+This file contains a FileIO implementation that relies on the filesystem interface provided
+by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
+type to use. Theoretically, this allows the supported storage types to grow naturally
+with the pyarrow library.
+"""
+
+import os
+from typing import Union
+from urllib.parse import urlparse
+
+from pyarrow.fs import FileSystem, FileType
+
+from iceberg.io.base import FileIO, InputFile, InputStream, OutputFile, OutputStream
+
+_FILESYSTEM_INSTANCES: dict = {}
+
+
+def get_filesystem(location: str):
+    """Retrieve a pyarrow.fs.FileSystem instance
+
+    This method checks _FILESYSTEM_INSTANCES for an existing filesystem based on the location's
+    scheme, i.e. s3, hdfs, viewfs. If an existing filesystem has not been cached, it instantiates a new
+    filesystem using `pyarrow.fs.FileSystem.from_uri(location)`, caches the returned filesystem, and
+    also returns that filesystem. If a path with no scheme is provided, it's assumed to be a path to
+    a local file.
+
+    Args:
+        location(str): The location of the file
+
+    Returns:
+        pyarrow.fs.FileSystem: An implementation of the FileSystem base class inferred from the location
+
+    Raises:
+        ArrowInvalid: A suitable FileSystem implementation cannot be found based on the location provided
+    """
+    parsed_location = urlparse(location)  # Create a ParseResult from the uri
+    if not parsed_location.scheme:  # If no scheme, assume the path is to a local file
+        if _FILESYSTEM_INSTANCES.get("local"):
+            filesystem = _FILESYSTEM_INSTANCES["local"]
+        else:
+            filesystem, _ = FileSystem.from_uri(os.path.abspath(location))
+            _FILESYSTEM_INSTANCES["local"] = filesystem
+    elif _FILESYSTEM_INSTANCES.get(parsed_location.scheme):  # Check for a cached filesystem

Review comment:
       I don't think Scheme is enough if we want this to be effective.  I believe 's3://` the first path component is bucket which implies different connections.  Based on Weston's response we would either want to eat the cost of create the file system each time or at least specialized for S3 to cache based on bucket.




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