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

PARQUET-1599: Fix to-avro to respect the overwrite option #650

Merged
merged 6 commits into from Apr 11, 2020
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -29,6 +29,7 @@
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.io.DatumWriter;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.cli.BaseCommand;
Expand Down Expand Up @@ -96,9 +97,8 @@ public int run() throws IOException {

Path outPath = qualifiedPath(outputPath);
FileSystem outFS = outPath.getFileSystem(getConf());
if (overwrite && outFS.exists(outPath)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SchemaCommand has the same problem.

Instead of throwing an exception here if the file already exists, how about adding a new method to BaseCommand for no overwrite case, and using that when creating the file in this command (and in SchemaCommand)?

Something like this: a new createWithNoOverwrite, which calls the private create with an extra overwrite=false flag, and it will use fs.create with this parameter value. How does this sound? We can also remove this delete file section in this case, and just call createWithNoOverwrite when overwrite is true, and call create otherwise.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the review @nandorKollar, that sounds good to me.
I'll update the PR a bit later.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@nandorKollar Sorry for my long silence, just addressed your comments :)

console.debug("Deleting output file {} (already exists)", outPath);
outFS.delete(outPath);
if (!overwrite && outFS.exists(outPath)) {
throw new FileAlreadyExistsException("Output file " + outputPath + " already exists");
}

Iterable<Record> reader = openDataFile(source, projection);
Expand Down
Expand Up @@ -26,14 +26,19 @@

public class AvroFileTest extends ParquetFileTest {

protected File toAvro(File parquetFile) throws IOException {
protected File toAvro(File inputFile) throws IOException {
File outputFile = new File(getTempFolder(), getClass().getSimpleName() + ".avro");
return toAvro(inputFile, outputFile, false);
}

protected File toAvro(File inputFile, File outputFile, boolean overwrite) throws IOException {
ToAvroCommand command = new ToAvroCommand(createLogger());
command.targets = Arrays.asList(parquetFile.getAbsolutePath());
File output = new File(getTempFolder(), getClass().getSimpleName() + ".avro");
command.outputPath = output.getAbsolutePath();
command.targets = Arrays.asList(inputFile.getAbsolutePath());
command.outputPath = outputFile.getAbsolutePath();
command.overwrite = overwrite;
command.setConf(new Configuration());
int exitCode = command.run();
assert(exitCode == 0);
return output;
return outputFile;
}
}
Expand Up @@ -18,6 +18,8 @@
*/
package org.apache.parquet.cli.commands;

import org.apache.commons.io.FileUtils;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.junit.Assert;
import org.junit.Test;

Expand All @@ -30,4 +32,20 @@ public void testToAvroCommand() throws IOException {
File avroFile = toAvro(parquetFile());
Assert.assertTrue(avroFile.exists());
}

@Test
public void testToAvroCommandOverwriteExistentFile() throws IOException {
File outputFile = new File(getTempFolder(), getClass().getSimpleName() + ".avro");
FileUtils.touch(outputFile);
Assert.assertEquals(0, outputFile.length());
File avroFile = toAvro(parquetFile(), outputFile, true);
Assert.assertTrue(0 < avroFile.length());
}

@Test(expected = FileAlreadyExistsException.class)
public void testToAvroCommandOverwriteExistentFileWithoutOverwriteOption() throws IOException {
File outputFile = new File(getTempFolder(), getClass().getSimpleName() + ".avro");
FileUtils.touch(outputFile);
toAvro(parquetFile(), outputFile, false);
}
}