-
Notifications
You must be signed in to change notification settings - Fork 3k
Added FileIO Support for ORC Reader and Writers #6293
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
| assertNotNull(is); | ||
|
|
||
| // Cannot use the filesystem for any other operation | ||
| assertThrows( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In general it would be good to always check against a specific error message. We typically use Assertions.assertThatThrownBy(() -> codeThatFails()).isInstanceOf(Xyz.class).hasMessage(...) to do that
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. Changing
| // Show that FileSystem access is not possible for the file we are supplying as the scheme | ||
| // dummy is not handled | ||
| ProxyOutputFile outFile = new ProxyOutputFile(Files.localOutput(temp.newFile())); | ||
| Assertions.assertThrows( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it would be better to move such assertions to AssertJ, because otherwise it'll be more work potentially to move such code when upgrading to Junit5
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense. Changing.
|
I'm not a big fan of the fake filesystem approach here, mostly because i'm afraid of mocking an object like that when we don't have the full filesystem state. I feel like this patch would have us maintaining a rather large Hadoop mock. Is there any chance we can convince the ORC project to allow the creation of a writer from an "java.io.OutputStream" instead of always creating its own file? |
|
|
||
| @Override | ||
| public FSDataInputStream open(Path f) throws IOException { | ||
| return open(f, 0); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Rather than making up a fake buffer size, I think it would be better to call this method from open(Path, int) and discard the buffer size that's passed into that one.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agreed this is sleek. Will change.
|
|
||
| @Override | ||
| public FSDataOutputStream create(Path f) throws IOException { | ||
| return create(f, null, true, 0, (short) 0, 0, null); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Like above, I this is the method to implement, not create(...).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agreed
| } | ||
|
|
||
| @Override | ||
| public FSDataOutputStream create(Path f) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should this use a Preconditions to check that the path to open is the same as outputFile.location()?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We do have that check, with the previous comment the check will now move to `create(Path f, boolean overwrite) method with the other method calling this.
| } | ||
| } | ||
|
|
||
| private static class NullFileSystem extends FileSystem { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there a better place for this fake FS stuff? Maybe a top-level util class?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure. Moved to FileIOFSUtil class
| @Override | ||
| protected void finalize() throws Throwable { | ||
| super.finalize(); | ||
| if (!closed) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think there's a need for this since the input stream should have its own finalizer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense. Removing this.
| if (file instanceof HadoopInputFile) { | ||
| readerOptions.filesystem(((HadoopInputFile) file).getFileSystem()); | ||
| } else { | ||
| readerOptions.filesystem(new InputFileSystem(file)).maxLength(file.getLength()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maxLength is used to avoid calls to get status? If so, we should add a comment.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added comment to clarify this.
| ReaderOptions readerOptions = OrcFile.readerOptions(config).useUTCTimestamp(true); | ||
| if (file instanceof HadoopInputFile) { | ||
| readerOptions.filesystem(((HadoopInputFile) file).getFileSystem()); | ||
| } else { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The newFileReader(String, ReaderOptions) above should be deprecated, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I moved the OrcFile.createReader method into the new method and deprecated the previous one. That is a package local method, if acceptable I can remove it as it has no other uses.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, rather than deprecating we should just remove it if it isn't public.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
deleted
| private boolean closed; | ||
| private final StackTraceElement[] createStack; | ||
|
|
||
| private WrappedSeekableInputStream(SeekableInputStream inputStream) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you move this stream to HadoopStreams? There's nothing ORC specific here and we may find other uses for it. That would also allow us to update the wrap methods that convert to detect double wrapping and return the underlying stream.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Did the following:
- Moved class to HadoopStreams
- Made HadoopStreams public to be accessible here
- Implemented DelegatingInputStream
| options.setSchema(orcSchema); | ||
| this.writer = newOrcWriter(file, options, metadata); | ||
|
|
||
| this.writer = ORC.newFileWriter(file, options, metadata); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
is the options.fileSystem call still needed above?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes that is still needed. We continue to retain the current behavior for HadoopIO. So the FileSystem from HadoopOutputFile or HadoopInputFile is used for the FS operations. When we use anything other than HadoopIO the FileIOFS kicks in.
| .hasMessageStartingWith("Failed to get file system for path: dummy"); | ||
|
|
||
| // We are creating the proxy | ||
| SortOrder sortOrder = SortOrder.builderFor(SCHEMA).withOrderId(10).asc("id").build(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think the sort order is needed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, removed it to simplify the test
| .schema(SCHEMA) | ||
| .createWriterFunc(GenericOrcWriter::buildWriter) | ||
| .overwrite() | ||
| .withSpec(PartitionSpec.unpartitioned()) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be the default, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry didn't follow this. What do we mean by default?
Before the patch if you gave a Local[Output|Input]File this was converted to FileSystem operation resulting in a LocalFileSystem for handling this.
To ensure that is not taking place, we are mimicking a schema dummy that is not handled, so if any FS operations happen normally then it will fail with Failed to get file system for path: dummy but if handled via FileIOFS then it should be successful.
I will add this comment to the test to make it clearer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to add withSpec or will it use the unpartitioned spec by default?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah we need it, without it I get an error that Cannot create data writer without spec
|
Looks mostly good overall. Thanks for getting this working @pavibhai! |
Thanks for this insight. Overall I agree that this should not be our long term answer and we shall work in the ORC community to make this integration better. |
|
|
||
| @SuppressWarnings("checkstyle:AbbreviationAsWordInName") | ||
| public class ORC { | ||
| private static final Logger LOG = LoggerFactory.getLogger(ORC.class); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I don't see any usage for this new Logger introduced by this patch.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice, catch. Removing this.
| import org.apache.iceberg.io.OutputFile; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
|
||
| public class FileIOFSUtil { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think this should be public. Could you make it package-private?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, done.
|
Thanks, @pavibhai! Great to have this fixed, even if it's a hack 😅. |
What?
This adds support for FileIO with Apache ORC by wrapping InputFile and OutputFile into a consumable FileSystem objects.
Why?
In the absence of this ORC is not able to leverage the benefits of better FileIO such as S3FileIO compared to S3AFileSystem
Tested?
New unit tests have been added to ensure the use of FileIO in ORC Readers and Writers