Skip to content
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

File I/O Submodule for TableOperations #12

Closed
mccheah opened this issue Nov 27, 2018 · 0 comments
Closed

File I/O Submodule for TableOperations #12

mccheah opened this issue Nov 27, 2018 · 0 comments

Comments

@mccheah
Copy link
Contributor

mccheah commented Nov 27, 2018

In Netflix/iceberg#107 it was discussed that InputFile and OutputFile instances should be pluggable. We discussed the fact that provision of InputFile and OutputFile instances should be handled by the TableOperations API. However, the Spark data source in particular only uses HadoopInputFile#fromPath for reading and HadoopOutputFile#fromPath for writing. Using TableOperations#newInputFile and TableOperations#newOutputFile, would also be difficult because calling these methods on the executors would require TableOperations instances to be Serializable.

We propose having the TableOperations API provide a FileIO module that handles the narrow role of reading, creating / writing, and deleting files. We propose the following:

interface FileIO extends Serializable {
  InputFile newInputFile(String path);
  OutputFile newOutputFile(String path);
  void deleteFile(String path);
}

Then the following method would be added to TableOperations, and we would remove TableOperations#newInputFile and TableOperations#newMetadataFile.

interface TableOperations {
  FileIO fileIo();
  String resolveNewMetadataPath(String metadataFilename);
}

The need for resolveNewMetadataPath is because the new FileIO abstraction considers all locations as full paths, but the old method TableOperations#newMetadataFile assumes the argument is a file name, not a full path. Therefore now callers that used to call TableOperations#newMetadataFile should first retrieve the full path and then pass that along to FileIO#newOutputFile. For convenience we could add a helper default method like so:

interface TableOperations {
  FileIO fileIo();
  String resolveNewMetadataPath(String metadataFilename);
  default OutputFile newMetadataFile(String fileName) {
    return fileIo().newOutputFile(resolveMetadataPath(fileName));
  }
}
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

1 participant