Skip to content

Conversation

@pavibhai
Copy link
Contributor

@pavibhai pavibhai commented Nov 28, 2022

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

assertNotNull(is);

// Cannot use the filesystem for any other operation
assertThrows(
Copy link
Contributor

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

Copy link
Contributor Author

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(
Copy link
Contributor

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense. Changing.

@RussellSpitzer
Copy link
Member

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);
Copy link
Contributor

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.

Copy link
Contributor Author

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);
Copy link
Contributor

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

Copy link
Contributor Author

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 {
Copy link
Contributor

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()?

Copy link
Contributor Author

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 {
Copy link
Contributor

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?

Copy link
Contributor Author

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) {
Copy link
Contributor

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.

Copy link
Contributor Author

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());
Copy link
Contributor

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.

Copy link
Contributor Author

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 {
Copy link
Contributor

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?

Copy link
Contributor Author

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.

Copy link
Contributor

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.

Copy link
Contributor Author

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) {
Copy link
Contributor

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.

Copy link
Contributor Author

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);
Copy link
Contributor

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?

Copy link
Contributor Author

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();
Copy link
Contributor

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.

Copy link
Contributor Author

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())
Copy link
Contributor

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?

Copy link
Contributor Author

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.

Copy link
Contributor

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?

Copy link
Contributor Author

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

@rdblue
Copy link
Contributor

rdblue commented Dec 7, 2022

Looks mostly good overall. Thanks for getting this working @pavibhai!

@pavibhai
Copy link
Contributor Author

pavibhai commented Dec 7, 2022

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?

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.
In the meantime given the relative stability of the FileSystem APIs and the limited exposure to just the creation of the input and output streams I hope this is not too painful to maintain.

@github-actions github-actions bot added the core label Dec 7, 2022
@pavibhai
Copy link
Contributor Author

pavibhai commented Dec 7, 2022

Looks mostly good overall. Thanks for getting this working @pavibhai!

Thanks @rdblue for your comments. I have addressed the comments, there are a few comments where I gave additional clarification.


@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class ORC {
private static final Logger LOG = LoggerFactory.getLogger(ORC.class);
Copy link
Collaborator

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.

Copy link
Contributor Author

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 {
Copy link
Contributor

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, done.

@rdblue rdblue merged commit b4d9770 into apache:master Dec 18, 2022
@rdblue
Copy link
Contributor

rdblue commented Dec 18, 2022

Thanks, @pavibhai! Great to have this fixed, even if it's a hack 😅.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants