From 928d35bdc30fcdf49f6edb8fa8d1e9da96361d71 Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Thu, 3 Aug 2023 23:28:06 +0900 Subject: [PATCH] PARQUET-2328: Add overwrite option to the parquet-cli's rewrite subcommand (#1125) --- .../parquet/cli/commands/RewriteCommand.java | 23 ++++++++++++-- .../cli/commands/RewriteCommandTest.java | 30 +++++++++++++++++++ 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/RewriteCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/RewriteCommand.java index c1dddd2993..da0b3eff4e 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/RewriteCommand.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/RewriteCommand.java @@ -22,6 +22,7 @@ import com.beust.jcommander.Parameters; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.cli.BaseCommand; import org.apache.parquet.hadoop.metadata.CompressionCodecName; @@ -44,16 +45,25 @@ public class RewriteCommand extends BaseCommand { description = "", required = true) List inputs; + @Parameter( names = {"-o", "--output"}, description = "", required = true) String output; + + @Parameter( + names={"--overwrite"}, + description="Overwrite the output file if it exists", + required = false) + boolean overwrite; + @Parameter( names = {"--mask-mode"}, description = "", required = false) String maskMode; + @Parameter( names = {"--mask-columns"}, description = "", @@ -76,7 +86,7 @@ public RewriteCommand(Logger console) { super(console); } - private RewriteOptions buildOptionsOrFail() { + private RewriteOptions buildOptionsOrFail() throws IOException { Preconditions.checkArgument(inputs != null && !inputs.isEmpty() && output != null, "Both input and output parquet file paths are required."); @@ -108,7 +118,16 @@ private RewriteOptions buildOptionsOrFail() { builder.transform(codecName); } - return builder.build(); + RewriteOptions options = builder.build(); + + // If RewriteOptions are successfully built and the overwrite option is specified, remove the output path + FileSystem outFS = outputPath.getFileSystem(getConf()); + if (overwrite && outFS.exists(outputPath)) { + console.debug("Deleting output file {} (already exists)", outputPath); + outFS.delete(outputPath); + } + + return options; } @Override diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/RewriteCommandTest.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/RewriteCommandTest.java index 2d87f8fd60..64212a8ca9 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/RewriteCommandTest.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/RewriteCommandTest.java @@ -19,11 +19,13 @@ package org.apache.parquet.cli.commands; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileAlreadyExistsException; import org.junit.Assert; import org.junit.Test; import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; public class RewriteCommandTest extends ParquetFileTest { @@ -38,4 +40,32 @@ public void testRewriteCommand() throws IOException { Assert.assertEquals(0, command.run()); Assert.assertTrue(output.exists()); } + + @Test(expected = FileAlreadyExistsException.class) + public void testRewriteCommandWithoutOverwrite() throws IOException { + File file = parquetFile(); + RewriteCommand command = new RewriteCommand(createLogger()); + command.inputs = Arrays.asList(file.getAbsolutePath()); + File output = new File(getTempFolder(), "converted.parquet"); + command.output = output.getAbsolutePath(); + command.setConf(new Configuration()); + + Files.createFile(output.toPath()); + command.run(); + } + + @Test + public void testRewriteCommandWithOverwrite() throws IOException { + File file = parquetFile(); + RewriteCommand command = new RewriteCommand(createLogger()); + command.inputs = Arrays.asList(file.getAbsolutePath()); + File output = new File(getTempFolder(), "converted.parquet"); + command.output = output.getAbsolutePath(); + command.overwrite = true; + command.setConf(new Configuration()); + + Files.createFile(output.toPath()); + Assert.assertEquals(0, command.run()); + Assert.assertTrue(output.exists()); + } }