You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by fo...@apache.org on 2019/10/07 09:24:56 UTC
[parquet-mr] branch master updated: PARQUET-1596: PARQUET-1375
broke parquet-cli's to-avro command (#648)
This is an automated email from the ASF dual-hosted git repository.
fokko 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 10f57a3 PARQUET-1596: PARQUET-1375 broke parquet-cli's to-avro command (#648)
10f57a3 is described below
commit 10f57a3779264ba222288defd1472d66ac2ae135
Author: Fokko Driesprong <fo...@apache.org>
AuthorDate: Mon Oct 7 11:24:51 2019 +0200
PARQUET-1596: PARQUET-1375 broke parquet-cli's to-avro command (#648)
* PARQUET-1596: PARQUET-1375 broke parquet-cli's to-avro command
The expected NPE:
cat /Users/fokkodriesprong/Desktop/parquet-mr/parquet-cli/target/surefire-reports/org.apache.parquet.cli.commands.ToAvroCommandTest.txt
-------------------------------------------------------------------------------
Test set: org.apache.parquet.cli.commands.ToAvroCommandTest
-------------------------------------------------------------------------------
Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 0.154 sec <<< FAILURE!
testToAvroCommandFromJson(org.apache.parquet.cli.commands.ToAvroCommandTest) Time elapsed: 0.052 sec <<< ERROR!
java.lang.NullPointerException
at org.apache.hadoop.fs.FileSystem.getDefaultUri(FileSystem.java:180)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:361)
at org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:344)
at org.apache.parquet.cli.BaseCommand.defaultFS(BaseCommand.java:81)
at org.apache.parquet.cli.BaseCommand.qualifiedPath(BaseCommand.java:164)
at org.apache.parquet.cli.BaseCommand.openSeekable(BaseCommand.java:215)
at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:375)
at org.apache.parquet.cli.commands.ToAvroCommand.run(ToAvroCommand.java:93)
at org.apache.parquet.cli.commands.ToAvroCommandTest.testToAvroCommandFromJson(ToAvroCommandTest.java:72)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
at org.junit.rules.RunRules.evaluate(RunRules.java:20)
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
at org.apache.maven.surefire.junit4.JUnit4TestSet.execute(JUnit4TestSet.java:53)
at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:123)
at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:104)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.apache.maven.surefire.util.ReflectionUtils.invokeMethodWithArray(ReflectionUtils.java:164)
at org.apache.maven.surefire.booter.ProviderFactory$ProviderProxy.invoke(ProviderFactory.java:110)
at org.apache.maven.surefire.booter.SurefireStarter.invokeProvider(SurefireStarter.java:175)
at org.apache.maven.surefire.booter.SurefireStarter.runSuitesInProcessWhenForked(SurefireStarter.java:107)
at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:68)
---
.../java/org/apache/parquet/cli/BaseCommand.java | 4 ++-
.../apache/parquet/cli/commands/ToAvroCommand.java | 22 ++++++------
.../java/org/apache/parquet/cli/json/AvroJson.java | 4 ++-
.../parquet/cli/commands/ToAvroCommandTest.java | 40 +++++++++++++++++++++-
pom.xml | 2 +-
5 files changed, 57 insertions(+), 15 deletions(-)
diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/BaseCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/BaseCommand.java
index 1bd74bd..cdef53d 100644
--- a/parquet-cli/src/main/java/org/apache/parquet/cli/BaseCommand.java
+++ b/parquet-cli/src/main/java/org/apache/parquet/cli/BaseCommand.java
@@ -226,7 +226,9 @@ public abstract class BaseCommand implements Command, Configurable {
@Override
public Configuration getConf() {
- return conf;
+ // In case conf is null, we'll return an empty configuration
+ // this can be on a local development machine
+ return null != conf ? conf : new Configuration();
}
/**
diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ToAvroCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ToAvroCommand.java
index ceb11cf..d659109 100644
--- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ToAvroCommand.java
+++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ToAvroCommand.java
@@ -86,30 +86,30 @@ public class ToAvroCommand extends BaseCommand {
CodecFactory codecFactory = Codecs.avroCodec(compressionCodecName);
- Schema schema;
+ final Schema schema;
if (avroSchemaFile != null) {
schema = Schemas.fromAvsc(open(avroSchemaFile));
} else {
schema = getAvroSchema(source);
}
- Schema projection = filterSchema(schema, columns);
+ final Schema projection = filterSchema(schema, columns);
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 (FileSystem outFS = outPath.getFileSystem(getConf())) {
+ if (overwrite && outFS.exists(outPath)) {
+ console.debug("Deleting output file {} (already exists)", outPath);
+ outFS.delete(outPath);
+ }
}
Iterable<Record> reader = openDataFile(source, projection);
boolean threw = true;
long count = 0;
- try {
- DatumWriter<Record> datumWriter = new GenericDatumWriter<>(schema);
- DataFileWriter<Record> w = new DataFileWriter<>(datumWriter);
- w.setCodec(codecFactory);
- try (DataFileWriter<Record> writer = w.create(projection, create(outputPath))) {
+ 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))) {
for (Record record : reader) {
writer.append(record);
count += 1;
diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/json/AvroJson.java b/parquet-cli/src/main/java/org/apache/parquet/cli/json/AvroJson.java
index 9f4756d..f67b99f 100644
--- a/parquet-cli/src/main/java/org/apache/parquet/cli/json/AvroJson.java
+++ b/parquet-cli/src/main/java/org/apache/parquet/cli/json/AvroJson.java
@@ -64,7 +64,9 @@ public class AvroJson {
private static final JsonFactory FACTORY = new JsonFactory(MAPPER);
public static Iterator<JsonNode> parser(final InputStream stream) {
- try(JsonParser parser = FACTORY.createParser(stream)) {
+ try {
+ // Don't close the parser until the iterator has been consumed
+ JsonParser parser = FACTORY.createParser(stream);
return parser.readValuesAs(JsonNode.class);
} catch (IOException e) {
throw new RuntimeIOException("Cannot read from stream", e);
diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ToAvroCommandTest.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ToAvroCommandTest.java
index 62d164d..9344a78 100644
--- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ToAvroCommandTest.java
+++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ToAvroCommandTest.java
@@ -16,22 +16,60 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.parquet.cli.commands;
+import com.beust.jcommander.JCommander;
import org.junit.Assert;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import java.io.BufferedWriter;
import java.io.File;
+import java.io.FileWriter;
import java.io.IOException;
public class ToAvroCommandTest extends AvroFileTest {
+ @Rule
+ public TemporaryFolder folder = new TemporaryFolder();
+
@Test
- public void testToAvroCommand() throws IOException {
+ public void testToAvroCommandFromParquet() throws IOException {
File avroFile = toAvro(parquetFile());
Assert.assertTrue(avroFile.exists());
}
@Test
+ public void testToAvroCommandFromJson() throws IOException {
+ final File jsonInputFile = folder.newFile("sample.json");
+ final File avroOutputFile = folder.newFile("sample.avro");
+
+ // Write the json to the file, so we can read it again.
+ final String inputJson = "{\"id\": 1, \"name\": \"Alice\"}\n" +
+ "{\"id\": 2, \"name\": \"Bob\"}\n" +
+ "{\"id\": 3, \"name\": \"Carol\"}\n" +
+ "{\"id\": 4, \"name\": \"Dave\"}";
+
+ try (BufferedWriter writer = new BufferedWriter(new FileWriter(jsonInputFile))) {
+ writer.write(inputJson);
+ }
+
+ ToAvroCommand cmd = new ToAvroCommand(null);
+
+ JCommander
+ .newBuilder()
+ .addObject(cmd)
+ .build()
+ .parse(
+ jsonInputFile.getAbsolutePath(),
+ "--output",
+ avroOutputFile.getAbsolutePath()
+ );
+
+ assert (cmd.run() == 0);
+ }
+
public void testToAvroCommandWithGzipCompression() throws IOException {
File avroFile = toAvro(parquetFile(), "GZIP");
Assert.assertTrue(avroFile.exists());
diff --git a/pom.xml b/pom.xml
index 445836f..daea619 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,7 +105,7 @@
<!-- parquet-cli dependencies -->
<opencsv.version>2.3</opencsv.version>
- <jcommander.version>1.35</jcommander.version>
+ <jcommander.version>1.72</jcommander.version>
<zstd-jni.version>1.4.0-1</zstd-jni.version>
</properties>