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 all commits
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 @@ -140,15 +140,37 @@ public FSDataOutputStream createWithChecksum(String filename)
return create(filename, false);
}

/**
* Creates a file and returns an open {@link FSDataOutputStream}.
*
* If the file does not have a file system scheme, this uses the default FS.
*
* This will neither produce checksum files nor overwrite a file that already
* exists.
*
* @param filename The filename to create
* @return An open FSDataOutputStream
* @throws IOException if there is an error creating the file
*/
public FSDataOutputStream createWithNoOverwrite(String filename)
throws IOException {
return create(filename, true, false);
}

private FSDataOutputStream create(String filename, boolean noChecksum)
throws IOException {
return create(filename, noChecksum, true);
}

private FSDataOutputStream create(String filename, boolean noChecksum, boolean overwrite)
throws IOException {
Path filePath = qualifiedPath(filename);
// even though it was qualified using the default FS, it may not be in it
FileSystem fs = filePath.getFileSystem(getConf());
if (noChecksum && fs instanceof ChecksumFileSystem) {
fs = ((ChecksumFileSystem) fs).getRawFileSystem();
}
return fs.create(filePath, true /* overwrite */);
return fs.create(filePath, overwrite);
}

/**
Expand Down
Expand Up @@ -26,7 +26,6 @@
import org.apache.parquet.cli.BaseCommand;
import org.apache.parquet.cli.util.Formats;
import org.apache.avro.file.SeekableInput;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
Expand Down Expand Up @@ -80,14 +79,8 @@ public int run() throws IOException {
String source = targets.get(0);

if (outputPath != null) {
Path outPath = qualifiedPath(outputPath);
FileSystem outFS = outPath.getFileSystem(getConf());
if (overwrite && outFS.exists(outPath)) {
console.debug("Deleting output file {} (already exists)", outPath);
outFS.delete(outPath);
}

try (OutputStream out = create(outputPath)) {
try (OutputStream out = overwrite ?
create(outputPath) : createWithNoOverwrite(outputPath)) {
out.write(getSchema(source).getBytes(StandardCharsets.UTF_8));
}
} else {
Expand Down
Expand Up @@ -29,14 +29,14 @@
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.io.DatumWriter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.cli.BaseCommand;
import org.apache.parquet.cli.util.Codecs;
import org.apache.parquet.cli.util.Schemas;
import org.slf4j.Logger;
import java.io.Closeable;
import java.io.IOException;
import java.io.OutputStream;
import java.util.List;

import static org.apache.avro.generic.GenericData.Record;
Expand Down Expand Up @@ -92,24 +92,17 @@ public int run() throws IOException {
} else {
schema = getAvroSchema(source);
}
final Schema projection = filterSchema(schema, columns);

Path outPath = qualifiedPath(outputPath);
try (FileSystem outFS = outPath.getFileSystem(getConf())) {
if (overwrite && outFS.exists(outPath)) {
console.debug("Deleting output file {} (already exists)", outPath);
outFS.delete(outPath);
}
}

final Schema projection = filterSchema(schema, columns);
Iterable<Record> reader = openDataFile(source, projection);
boolean threw = true;
long count = 0;

DatumWriter<Record> datumWriter = new GenericDatumWriter<>(schema);
try (DataFileWriter<Record> fileWriter = new DataFileWriter<>(datumWriter)) {
fileWriter.setCodec(codecFactory);
try (DataFileWriter<Record> writer=fileWriter.create(projection, create(outputPath))) {
try (OutputStream os = overwrite ? create(outputPath) : createWithNoOverwrite(outputPath);
DataFileWriter<Record> writer = fileWriter.create(projection, os)) {
for (Record record : reader) {
writer.append(record);
count += 1;
Expand Down
Expand Up @@ -26,19 +26,28 @@

public class AvroFileTest extends ParquetFileTest {

protected File toAvro(File parquetFile) throws IOException {
return toAvro(parquetFile, "GZIP");
protected File toAvro(File inputFile) throws IOException {
return toAvro(inputFile, "GZIP");
}

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

protected File toAvro(File inputFile, File outputFile, boolean overwrite) throws IOException {
return toAvro(inputFile, outputFile, overwrite, "GZIP");
}

protected File toAvro(File inputFile, File outputFile, boolean overwrite, String compressionCodecName) 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.compressionCodecName = compressionCodecName;
command.overwrite = overwrite;
command.setConf(new Configuration());
int exitCode = command.run();
assert(exitCode == 0);
return output;
return outputFile;
}
}
Expand Up @@ -18,7 +18,9 @@
*/
package org.apache.parquet.cli.commands;

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

Expand All @@ -35,4 +37,32 @@ public void testSchemaCommand() throws IOException {
command.setConf(new Configuration());
Assert.assertEquals(0, command.run());
}

@Test
public void testSchemaCommandOverwriteExistentFile() throws IOException {
File inputFile = parquetFile();
File outputFile = new File(getTempFolder(), getClass().getSimpleName() + ".avsc");
FileUtils.touch(outputFile);
Assert.assertEquals(0, outputFile.length());
SchemaCommand command = new SchemaCommand(createLogger());
command.targets = Arrays.asList(inputFile.getAbsolutePath());
command.outputPath = outputFile.getAbsolutePath();
command.overwrite = true;
command.setConf(new Configuration());
Assert.assertEquals(0, command.run());
Assert.assertTrue(0 < outputFile.length());
}


@Test(expected = FileAlreadyExistsException.class)
public void testSchemaCommandOverwriteExistentFileWithoutOverwriteOption() throws IOException {
File inputFile = parquetFile();
File outputFile = new File(getTempFolder(), getClass().getSimpleName() + ".avsc");
FileUtils.touch(outputFile);
SchemaCommand command = new SchemaCommand(createLogger());
command.targets = Arrays.asList(inputFile.getAbsolutePath());
command.outputPath = outputFile.getAbsolutePath();
command.setConf(new Configuration());
command.run();
}
}
Expand Up @@ -20,6 +20,8 @@
package org.apache.parquet.cli.commands;

import com.beust.jcommander.JCommander;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
Expand Down Expand Up @@ -62,6 +64,7 @@ public void testToAvroCommandFromJson() throws IOException {
.addObject(cmd)
.build()
.parse(
"--overwrite",
jsonInputFile.getAbsolutePath(),
"--output",
avroOutputFile.getAbsolutePath()
Expand Down Expand Up @@ -91,4 +94,20 @@ public void testToAvroCommandWithZstdCompression() throws IOException {
public void testToAvroCommandWithInvalidCompression() throws IOException {
toAvro(parquetFile(), "FOO");
}

@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);
}
}