This is an automated email from the ASF dual-hosted git repository.

gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new 928d35bdc PARQUET-2328: Add overwrite option to the parquet-cli's 
rewrite subcommand (#1125)
928d35bdc is described below

commit 928d35bdc30fcdf49f6edb8fa8d1e9da96361d71
Author: Kengo Seki <[email protected]>
AuthorDate: Thu Aug 3 23:28:06 2023 +0900

    PARQUET-2328: Add overwrite option to the parquet-cli's rewrite subcommand 
(#1125)
---
 .../parquet/cli/commands/RewriteCommand.java       | 23 +++++++++++++++--
 .../parquet/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 c1dddd299..da0b3eff4 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.Parameter;
 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 = "<comma-separated text of input parquet file paths>",
           required = true)
   List<String> inputs;
+
   @Parameter(
           names = {"-o", "--output"},
           description = "<output parquet file path>",
           required = true)
   String output;
+
+  @Parameter(
+          names={"--overwrite"},
+          description="Overwrite the output file if it exists",
+          required = false)
+  boolean overwrite;
+
   @Parameter(
           names = {"--mask-mode"},
           description = "<mask mode: nullify>",
           required = false)
   String maskMode;
+
   @Parameter(
           names = {"--mask-columns"},
           description = "<columns to be replaced with masked value>",
@@ -76,7 +86,7 @@ public class RewriteCommand extends BaseCommand {
     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 @@ public class RewriteCommand extends BaseCommand {
       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 2d87f8fd6..64212a8ca 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 class RewriteCommandTest extends ParquetFileTest {
     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());
+  }
 }

Reply via email to