Skip to content

Commit 43c5976

Browse files
authored
GH-3300: add ParquetWriter and ParquetReader builders constructor without params (#3301)
1 parent c002328 commit 43c5976

File tree

5 files changed

+162
-1
lines changed

5 files changed

+162
-1
lines changed

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java

Lines changed: 25 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -190,9 +190,13 @@ public static <T> Builder<T> builder(ReadSupport<T> readSupport, Path path) {
190190
return new Builder<>(readSupport, path);
191191
}
192192

193+
public static <T> Builder<T> builder() {
194+
return new Builder<T>();
195+
}
196+
193197
public static class Builder<T> {
194198
private final ReadSupport<T> readSupport;
195-
private final InputFile file;
199+
private InputFile file;
196200
private final Path path;
197201
private Filter filter = null;
198202
private ByteBufferAllocator allocator = new HeapByteBufferAllocator();
@@ -223,6 +227,14 @@ protected Builder(Path path) {
223227
this.optionsBuilder = HadoopReadOptions.builder(this.conf, path);
224228
}
225229

230+
protected Builder() {
231+
this.readSupport = null;
232+
this.file = null;
233+
this.path = null;
234+
this.configuration = new HadoopParquetConfiguration();
235+
this.optionsBuilder = HadoopReadOptions.builder(this.configuration);
236+
}
237+
226238
protected Builder(InputFile file) {
227239
this.readSupport = null;
228240
this.file = Objects.requireNonNull(file, "file cannot be null");
@@ -250,6 +262,14 @@ protected Builder(InputFile file, ParquetConfiguration conf) {
250262
}
251263
}
252264

265+
public Builder<T> withFile(InputFile file) {
266+
this.file = Objects.requireNonNull(file, "file cannot be null");
267+
if (this.path != null) {
268+
throw new IllegalStateException("Path is already set");
269+
}
270+
return this;
271+
}
272+
253273
// when called, resets options to the defaults from conf
254274
public Builder<T> withConf(Configuration conf) {
255275
this.conf = Objects.requireNonNull(conf, "conf cannot be null");
@@ -384,6 +404,10 @@ protected ReadSupport<T> getReadSupport() {
384404
}
385405

386406
public ParquetReader<T> build() throws IOException {
407+
if (file == null && path == null) {
408+
throw new IllegalStateException("File or Path must be set");
409+
}
410+
387411
ParquetReadOptions options = optionsBuilder.withAllocator(allocator).build();
388412

389413
if (path != null) {

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.io.IOException;
2323
import java.util.HashMap;
2424
import java.util.Map;
25+
import java.util.Objects;
2526
import org.apache.hadoop.conf.Configuration;
2627
import org.apache.hadoop.fs.Path;
2728
import org.apache.parquet.bytes.ByteBufferAllocator;
@@ -476,6 +477,9 @@ public abstract static class Builder<T, SELF extends Builder<T, SELF>> {
476477
private boolean enableValidation = DEFAULT_IS_VALIDATING_ENABLED;
477478
private ParquetProperties.Builder encodingPropsBuilder = ParquetProperties.builder();
478479

480+
protected Builder() {}
481+
482+
@Deprecated
479483
protected Builder(Path path) {
480484
this.path = path;
481485
}
@@ -527,6 +531,20 @@ public SELF withConf(ParquetConfiguration conf) {
527531
return self();
528532
}
529533

534+
/**
535+
* Set the {@link OutputFile} to be written by the constructed writer.
536+
*
537+
* @param file a {@code OutputFile}
538+
* @return this builder for method chaining.
539+
*/
540+
public SELF withFile(OutputFile file) {
541+
this.file = Objects.requireNonNull(file, "file cannot be null");
542+
if (this.path != null) {
543+
throw new IllegalStateException("Cannot set both path and file");
544+
}
545+
return self();
546+
}
547+
530548
/**
531549
* Set the {@link ParquetFileWriter.Mode write mode} used when creating the
532550
* backing file for this writer.
@@ -960,6 +978,9 @@ public SELF withSizeStatisticsEnabled(boolean enabled) {
960978
* @throws IOException if there is an error while creating the writer
961979
*/
962980
public ParquetWriter<T> build() throws IOException {
981+
if (file == null && path == null) {
982+
throw new IllegalStateException("File or Path must be set");
983+
}
963984
if (conf == null) {
964985
conf = new HadoopParquetConfiguration();
965986
}

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,17 @@
3737
*/
3838
public class ExampleParquetWriter extends ParquetWriter<Group> {
3939

40+
/**
41+
* Creates a Builder without preconfigured Output File for configuring
42+
* ParquetWriter with the example object model.
43+
* THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE.
44+
*
45+
* @return a {@link Builder} to create a {@link ParquetWriter}
46+
*/
47+
public static Builder builder() {
48+
return new Builder();
49+
}
50+
4051
/**
4152
* Creates a Builder for configuring ParquetWriter with the example object
4253
* model. THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE.
@@ -106,6 +117,10 @@ private Builder(OutputFile file) {
106117
super(file);
107118
}
108119

120+
private Builder() {
121+
super();
122+
}
123+
109124
public Builder withType(MessageType type) {
110125
this.type = type;
111126
return this;

parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReader.java

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@
3333
import java.util.HashSet;
3434
import java.util.List;
3535
import java.util.Set;
36+
import java.util.concurrent.Callable;
3637
import org.apache.hadoop.conf.Configuration;
3738
import org.apache.hadoop.fs.Path;
3839
import org.apache.parquet.bytes.HeapByteBufferAllocator;
@@ -41,8 +42,12 @@
4142
import org.apache.parquet.example.data.Group;
4243
import org.apache.parquet.filter2.compat.FilterCompat;
4344
import org.apache.parquet.filter2.recordlevel.PhoneBookWriter;
45+
import org.apache.parquet.hadoop.ParquetReader.Builder;
46+
import org.apache.parquet.hadoop.api.ReadSupport;
4447
import org.apache.parquet.hadoop.example.ExampleParquetWriter;
4548
import org.apache.parquet.hadoop.example.GroupReadSupport;
49+
import org.apache.parquet.hadoop.util.HadoopInputFile;
50+
import org.apache.parquet.io.InputFile;
4651
import org.junit.After;
4752
import org.junit.AfterClass;
4853
import org.junit.Before;
@@ -230,4 +235,42 @@ public void testNoFiltering() throws Exception {
230235
assertEquals(DATA, readUsers(FilterCompat.NOOP, false, true));
231236
assertEquals(DATA, readUsers(FilterCompat.NOOP, true, true));
232237
}
238+
239+
private static class TestParquetReaderBuilder extends ParquetReader.Builder<Group> {
240+
241+
@Override
242+
protected ReadSupport<Group> getReadSupport() {
243+
return new GroupReadSupport();
244+
}
245+
}
246+
247+
@Test
248+
public void testParquetReaderBuilderWithInputFile() throws Exception {
249+
InputFile inputFile = HadoopInputFile.fromPath(file, new Configuration());
250+
Builder<Group> builder = new TestParquetReaderBuilder().withFile(inputFile);
251+
assertEquals(DATA, PhoneBookWriter.readUsers(builder, false));
252+
}
253+
254+
@Test
255+
public void testParquetReaderBuilderValidatesThatInputFileCanNotBeNull() throws Exception {
256+
TestUtils.assertThrows("file cannot be null", NullPointerException.class, (Callable<ParquetReader<Group>>)
257+
() -> new TestParquetReaderBuilder().withFile(null).build());
258+
}
259+
260+
@Test
261+
public void testParquetReaderBuilderValidatesThatInputFileIsSet() throws Exception {
262+
TestUtils.assertThrows("File or Path must be set", IllegalStateException.class, (Callable<ParquetReader<Group>>)
263+
() -> new TestParquetReaderBuilder().build());
264+
}
265+
266+
@Test
267+
public void testParquetReaderBuilderCanNotConfigurePathAndFile() throws Exception {
268+
TestUtils.assertThrows(
269+
"Path is already set", IllegalStateException.class, (Callable<ParquetReader<Group>>) () -> {
270+
InputFile inputFile = HadoopInputFile.fromPath(file, new Configuration());
271+
return ParquetReader.<Group>builder(new GroupReadSupport(), file)
272+
.withFile(inputFile)
273+
.build();
274+
});
275+
}
233276
}

parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java

Lines changed: 58 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -722,4 +722,62 @@ private void testV2WriteAllNullValues(
722722
}
723723
}
724724
}
725+
726+
@Test
727+
public void testParquetWriterConfiguringOutputFile() throws IOException {
728+
MessageType schema = Types.buildMessage()
729+
.required(BINARY)
730+
.as(stringType())
731+
.named("name")
732+
.named("msg");
733+
734+
Configuration conf = new Configuration();
735+
GroupWriteSupport.setSchema(schema, conf);
736+
737+
GroupFactory factory = new SimpleGroupFactory(schema);
738+
File file = temp.newFile();
739+
file.delete();
740+
Path path = new Path(file.getAbsolutePath());
741+
OutputFile outputFile = new TestOutputFile(path, conf);
742+
743+
String[] testNames = {"new", "writer", "builder", "without", "file"};
744+
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder()
745+
.withFile(outputFile)
746+
.withConf(conf)
747+
.build()) {
748+
for (String testName : testNames) {
749+
writer.write(factory.newGroup().append("name", testName));
750+
}
751+
}
752+
ParquetReader<Group> reader =
753+
ParquetReader.builder(new GroupReadSupport(), path).build();
754+
assertEquals("new", reader.read().getBinary("name", 0).toStringUsingUTF8());
755+
assertEquals("writer", reader.read().getBinary("name", 0).toStringUsingUTF8());
756+
assertEquals("builder", reader.read().getBinary("name", 0).toStringUsingUTF8());
757+
assertEquals("without", reader.read().getBinary("name", 0).toStringUsingUTF8());
758+
assertEquals("file", reader.read().getBinary("name", 0).toStringUsingUTF8());
759+
}
760+
761+
@Test
762+
public void testParquetWriterBuilderOutputFileCanNotBeNull() throws IOException {
763+
TestUtils.assertThrows("file cannot be null", NullPointerException.class, (Callable<ParquetWriter<Group>>)
764+
() -> ExampleParquetWriter.builder().withFile(null).build());
765+
}
766+
767+
@Test
768+
public void testParquetWriterBuilderValidatesThatOutputFileIsSet() throws IOException {
769+
TestUtils.assertThrows("File or Path must be set", IllegalStateException.class, (Callable<ParquetWriter<Group>>)
770+
() -> ExampleParquetWriter.builder().build());
771+
}
772+
773+
@Test
774+
public void testParquetWriterBuilderCanNotConfigurePathAndFile() throws IOException {
775+
File file = temp.newFile();
776+
Path path = new Path(file.getAbsolutePath());
777+
Configuration conf = new Configuration();
778+
OutputFile outputFile = new TestOutputFile(path, conf);
779+
TestUtils.assertThrows(
780+
"Cannot set both path and file", IllegalStateException.class, (Callable<ParquetWriter<Group>>) () ->
781+
ExampleParquetWriter.builder(path).withFile(outputFile).build());
782+
}
725783
}

0 commit comments

Comments
 (0)