-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-381: Add feature to merge metadata (summary) files, and control which files are generated #277
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
PARQUET-381: Add feature to merge metadata (summary) files, and control which files are generated #277
Changes from all commits
9d2b8da
87a2ebc
ddaf4ff
7a98957
099c913
96b9495
86232f5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,6 +30,7 @@ | |
| import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; | ||
|
|
||
| import org.apache.parquet.Log; | ||
| import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; | ||
| import org.apache.parquet.hadoop.util.ContextUtil; | ||
|
|
||
| public class ParquetOutputCommitter extends FileOutputCommitter { | ||
|
|
@@ -48,30 +49,63 @@ public void commitJob(JobContext jobContext) throws IOException { | |
| writeMetaDataFile(configuration,outputPath); | ||
| } | ||
|
|
||
| // TODO: This method should propagate errors, and we should clean up | ||
| // TODO: all the catching of Exceptions below -- see PARQUET-383 | ||
| public static void writeMetaDataFile(Configuration configuration, Path outputPath) { | ||
| if (configuration.getBoolean(ParquetOutputFormat.ENABLE_JOB_SUMMARY, true)) { | ||
| JobSummaryLevel level = ParquetOutputFormat.getJobSummaryLevel(configuration); | ||
| if (level == JobSummaryLevel.NONE) { | ||
| return; | ||
| } | ||
|
|
||
| try { | ||
| final FileSystem fileSystem = outputPath.getFileSystem(configuration); | ||
| FileStatus outputStatus = fileSystem.getFileStatus(outputPath); | ||
| List<Footer> footers; | ||
|
|
||
| switch (level) { | ||
| case ALL: | ||
| footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus, false); // don't skip row groups | ||
| break; | ||
| case COMMON_ONLY: | ||
| footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus, true); // skip row groups | ||
| break; | ||
| default: | ||
| throw new IllegalArgumentException("Unrecognized job summary level: " + level); | ||
| } | ||
|
|
||
| // If there are no footers, _metadata file cannot be written since there is no way to determine schema! | ||
| // Onus of writing any summary files lies with the caller in this case. | ||
| if (footers.isEmpty()) { | ||
| return; | ||
| } | ||
|
|
||
| try { | ||
| final FileSystem fileSystem = outputPath.getFileSystem(configuration); | ||
| FileStatus outputStatus = fileSystem.getFileStatus(outputPath); | ||
| List<Footer> footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus); | ||
| // If there are no footers, _metadata file cannot be written since there is no way to determine schema! | ||
| // Onus of writing any summary files lies with the caller in this case. | ||
| if (footers.isEmpty()) { | ||
| return; | ||
| } | ||
| ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers, level); | ||
| } catch (Exception e) { | ||
| LOG.warn("could not write summary file(s) for " + outputPath, e); | ||
|
|
||
| final Path metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE); | ||
|
|
||
| try { | ||
| ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers); | ||
| } catch (Exception e) { | ||
| LOG.warn("could not write summary file for " + outputPath, e); | ||
| final Path metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE); | ||
| if (fileSystem.exists(metadataPath)) { | ||
| fileSystem.delete(metadataPath, true); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. if the committer were to fail, I guess cleaning after would not be necessary?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure, do files written by the committer need to be deleted, or are they being written to a temp dir anyway? |
||
| } | ||
| } catch (Exception e2) { | ||
| LOG.warn("could not delete metadata file" + outputPath, e2); | ||
| } | ||
| } catch (Exception e) { | ||
| LOG.warn("could not write summary file for " + outputPath, e); | ||
|
|
||
| try { | ||
| final Path commonMetadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE); | ||
| if (fileSystem.exists(commonMetadataPath)) { | ||
| fileSystem.delete(commonMetadataPath, true); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. if previous fs operations failed, this would never be called either
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Looks also it fixes PARQUET-359 (see #258)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. haha nice. Yeah so we need to wrap the above deletion in a try / catch (though some of those method return boolean instead of throwing, let me check) |
||
| } | ||
| } catch (Exception e2) { | ||
| LOG.warn("could not delete metadata file" + outputPath, e2); | ||
| } | ||
|
|
||
| } | ||
| } catch (Exception e) { | ||
| LOG.warn("could not write summary file for " + outputPath, e); | ||
| } | ||
| } | ||
|
|
||
| } | ||
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.
since JobSummaryLevel is an enum, I guess it's really to guard against your own code than a user mistake.
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's to guard against adding new enum values to JobSummaryLevel, but not updating the places where it's used. Similar to having a default clause in all your switch statements. (it's too bad there aren't exhaustive switch statements).