ORC-2149: Supports merging multiple ORC files with the same schema into multiple ORC files.#2601
ORC-2149: Supports merging multiple ORC files with the same schema into multiple ORC files.#2601QianyongY wants to merge 1 commit intoapache:mainfrom
Conversation
There was a problem hiding this comment.
Pull request overview
Extends the ORC Java merge tool to optionally split merged output into multiple ORC part files under an output directory when --maxSize is provided, while keeping the existing single-output behavior by default.
Changes:
- Add
--maxSizeoption tomergeto batch inputs (by on-disk size) into multiple output part files. - Add a new unit test covering multi-part merge behavior.
- Update CLI help text and documentation to describe the new multi-output mode.
Reviewed changes
Copilot reviewed 4 out of 4 changed files in this pull request and generated 6 comments.
| File | Description |
|---|---|
site/_docs/java-tools.md |
Updates merge tool docs to describe single vs multi-output mode and examples. |
java/tools/src/java/org/apache/orc/tools/MergeFiles.java |
Implements --maxSize parsing and multi-part merge batching logic. |
java/tools/src/test/org/apache/orc/tools/TestMergeFiles.java |
Adds a unit test to validate multi-part output behavior. |
java/tools/src/java/org/apache/orc/tools/Driver.java |
Updates driver help text for the merge subcommand description. |
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
| long maxSizeBytes = 0; | ||
| if (cli.hasOption("maxSize")) { | ||
| maxSizeBytes = Long.parseLong(cli.getOptionValue("maxSize")); | ||
| if (maxSizeBytes <= 0) { | ||
| System.err.println("--maxSize must be a positive number of bytes."); | ||
| System.exit(1); | ||
| } | ||
| } |
There was a problem hiding this comment.
Long.parseLong(cli.getOptionValue("maxSize")) will throw NumberFormatException for non-numeric inputs and currently results in an uncaught exception/stack trace. Consider catching the parse error, printing a clear message (and help text if appropriate), and exiting with a non-zero status.
| for (int i = 0; i < batches.size(); i++) { | ||
| List<Path> batch = batches.get(i); | ||
| Path partOutput = new Path(outputDir, String.format(PART_FILE_FORMAT, i)); | ||
| List<Path> merged = OrcFile.mergeFiles(partOutput, OrcFile.writerOptions(conf), batch); |
There was a problem hiding this comment.
mergeIntoMultipleFiles takes writerOptions but ignores it and instead creates new OrcFile.writerOptions(conf) for each batch. This is confusing and can also drop caller-specified options (eg overwrite/key provider) if they are added later. Consider either removing the unused parameter, or (preferably) cloning the passed writerOptions for each batch (WriterOptions supports clone()) and passing the clone into OrcFile.mergeFiles.
| List<Path> merged = OrcFile.mergeFiles(partOutput, OrcFile.writerOptions(conf), batch); | |
| OrcFile.WriterOptions batchWriterOptions = writerOptions.clone(); | |
| List<Path> merged = OrcFile.mergeFiles(partOutput, batchWriterOptions, batch); |
| Path outputDir, | ||
| long maxSizeBytes) throws Exception { | ||
| FileSystem outFs = outputDir.getFileSystem(conf); | ||
| outFs.mkdirs(outputDir); |
There was a problem hiding this comment.
In multi-file mode the code calls outFs.mkdirs(outputDir) without validating what already exists at outputDir. If --output points to an existing file (or an existing non-empty directory), later writes may fail in confusing ways or partially succeed. Consider checking for outFs.exists(outputDir) / outFs.isFile(outputDir) and either failing fast with a clear message or requiring an empty directory.
| outFs.mkdirs(outputDir); | |
| if (outFs.exists(outputDir)) { | |
| if (!outFs.getFileStatus(outputDir).isDirectory()) { | |
| throw new IllegalArgumentException( | |
| "Output path already exists and is not a directory: " + outputDir); | |
| } | |
| if (outFs.listStatus(outputDir).length > 0) { | |
| throw new IllegalArgumentException( | |
| "Output directory must be empty for multi-file merge: " + outputDir); | |
| } | |
| } else if (!outFs.mkdirs(outputDir)) { | |
| throw new IllegalStateException("Failed to create output directory: " + outputDir); | |
| } |
| MergeFiles.main(conf, new String[]{workDir.toString(), | ||
| "--output", outputDir.toString(), | ||
| "--maxSize", String.valueOf(maxSize)}); | ||
| System.out.flush(); | ||
| System.setOut(origOut); |
There was a problem hiding this comment.
System.setOut(...) is not restored in a finally block. If MergeFiles.main throws, stdout will remain redirected and may break subsequent tests. Wrap the stdout redirection/restoration in try/finally (or use a helper/extension) so it is always restored.
| MergeFiles.main(conf, new String[]{workDir.toString(), | |
| "--output", outputDir.toString(), | |
| "--maxSize", String.valueOf(maxSize)}); | |
| System.out.flush(); | |
| System.setOut(origOut); | |
| try { | |
| MergeFiles.main(conf, new String[]{workDir.toString(), | |
| "--output", outputDir.toString(), | |
| "--maxSize", String.valueOf(maxSize)}); | |
| System.out.flush(); | |
| } finally { | |
| System.setOut(origOut); | |
| } |
| // Create 3 source ORC files with different row counts. | ||
| String[] sourceNames = { | ||
| workDir + File.separator + "ms-1.orc", | ||
| workDir + File.separator + "ms-2.orc", | ||
| workDir + File.separator + "ms-3.orc" | ||
| }; | ||
| int[] rowCounts = {5000, 5000, 5000}; | ||
| for (int f = 0; f < sourceNames.length; f++) { |
There was a problem hiding this comment.
The comment says the 3 source ORC files have "different row counts", but rowCounts is {5000, 5000, 5000}. Either update the comment or vary the row counts so the test matches its description.
| result.addOption(Option.builder("m") | ||
| .longOpt("maxSize") | ||
| .desc("Maximum size in bytes for each output ORC file. When set, --output is treated as " | ||
| + "an output directory and merged files are written as part-00000.orc, " | ||
| + "part-00001.orc, etc. Files are grouped at file boundaries so an individual " | ||
| + "file larger than this threshold will still be placed in its own part.") | ||
| .hasArg() | ||
| .build()); |
There was a problem hiding this comment.
--maxSize is enforced using the cumulative input file sizes (LocatedFileStatus#getLen()), not the actual size of the generated output ORC part files. The option description currently reads as if it caps the output file size; consider rewording to avoid implying a strict output-size limit.
|
Thank you for making a PR, @QianyongY . |
What changes were proposed in this pull request?
Extends the Java merge tool so that, for inputs sharing the same schema, you can still merge to one ORC file by default, or use -m / --maxSize to write multiple ORC files under an output directory as part-xxxxx.orc, batching by input file size.
Why are the changes needed?
Users often need to merge many compatible ORC files without producing a single huge output file. This adds an optional mode that caps output size at whole-file boundaries while keeping the existing single-file behavior when --maxSize is not set.
How was this patch tested?
Add UT