Skip to content

Commit a064890

Browse files
committed
PARQUET-2171. Vector IO: review comments
1 parent cc2c9df commit a064890

File tree

5 files changed

+38
-25
lines changed

5 files changed

+38
-25
lines changed

parquet-hadoop/pom.xml

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -79,11 +79,6 @@
7979
<version>${hadoop.version}</version>
8080
<scope>provided</scope>
8181
</dependency>
82-
<!-- <dependency>
83-
<groupId>org.apache.hadoop.extensions</groupId>
84-
<artifactId>hadoop-api-shim</artifactId>
85-
<scope>provided</scope>
86-
</dependency>-->
8782
<dependency>
8883
<groupId>org.apache.parquet</groupId>
8984
<artifactId>parquet-jackson</artifactId>

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/vectorio/BindingUtils.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@
3333
import org.apache.parquet.util.DynMethods;
3434

3535
/**
36-
* Package-private binding utils.
36+
* Binding utils.
3737
*/
3838
public final class BindingUtils {
3939
private static final Logger LOG = LoggerFactory.getLogger(BindingUtils.class);
@@ -127,9 +127,7 @@ public static IOException unwrapInnerException(final Throwable e) {
127127
} else if (cause instanceof UncheckedIOException) {
128128
// this is always an IOException
129129
return ((UncheckedIOException) cause).getCause();
130-
} else if (cause instanceof CompletionException) {
131-
return unwrapInnerException(cause);
132-
} else if (cause instanceof ExecutionException) {
130+
} else if (cause instanceof CompletionException || cause instanceof ExecutionException) {
133131
return unwrapInnerException(cause);
134132
} else if (cause instanceof RuntimeException) {
135133
throw (RuntimeException) cause;

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/vectorio/VectorIOBridge.java

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -168,11 +168,8 @@ public static void readVectoredRanges(
168168
ParquetFileRange parquetFileRange = (ParquetFileRange) fileRange.getReference();
169169
parquetFileRange.setDataReadFuture(fileRange.getData());
170170
});
171-
//throw new RuntimeException("readVectoredRanges");
172-
173171
}
174172

175-
176173
/**
177174
* Read data in a list of wrapped file ranges, extracting the inner
178175
* instances and then executing.

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

Lines changed: 35 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -110,6 +110,12 @@
110110
import org.slf4j.Logger;
111111
import org.slf4j.LoggerFactory;
112112

113+
/**
114+
* Verify that data can be written and read back again.
115+
* The test suite is parameterized on vector IO being disabled/enabled.
116+
* This verifies that the vector IO code path is correct, and that
117+
* the default path continues to work.
118+
*/
113119
@RunWith(Parameterized.class)
114120
public class TestParquetFileWriter {
115121

@@ -152,16 +158,24 @@ public static List<Boolean> params() {
152158
/**
153159
* Read type: true for vectored IO.
154160
*/
155-
private final boolean readType;
161+
private final boolean vectoredRead;
156162

157-
public TestParquetFileWriter(boolean readType) {
158-
this.readType = readType;
163+
/**
164+
* Instantiate.
165+
* @param vectoredRead use vector IO for reading.
166+
*/
167+
public TestParquetFileWriter(boolean vectoredRead) {
168+
this.vectoredRead = vectoredRead;
159169
}
160170

171+
/**
172+
* Get the configuration for the tests.
173+
* @return a configuration which may have vector IO set.
174+
*/
161175
private Configuration getTestConfiguration() {
162176
Configuration conf = new Configuration();
163177
// set the vector IO option
164-
conf.setBoolean(ParquetInputFormat.HADOOP_VECTORED_IO_ENABLED, readType);
178+
conf.setBoolean(ParquetInputFormat.HADOOP_VECTORED_IO_ENABLED, vectoredRead);
165179
return conf;
166180
}
167181

@@ -297,7 +311,7 @@ public void testWriteReadWithRecordReader() throws Exception {
297311
testFile.delete();
298312

299313
Path path = new Path(testFile.toURI());
300-
Configuration configuration = new Configuration();
314+
Configuration configuration = getTestConfiguration();
301315

302316
ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path);
303317
w.start();
@@ -395,7 +409,7 @@ public void testBloomFilterWriteRead() throws Exception {
395409
File testFile = temp.newFile();
396410
testFile.delete();
397411
Path path = new Path(testFile.toURI());
398-
Configuration configuration = new Configuration();
412+
Configuration configuration = getTestConfiguration();
399413
configuration.set("parquet.bloom.filter.column.names", "foo");
400414
String[] colPath = {"foo"};
401415
ColumnDescriptor col = schema.getColumnDescription(colPath);
@@ -436,7 +450,7 @@ public void testWriteReadDataPageV2() throws Exception {
436450
testFile.delete();
437451

438452
Path path = new Path(testFile.toURI());
439-
Configuration configuration = new Configuration();
453+
Configuration configuration = getTestConfiguration();
440454

441455
ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path);
442456
w.start();
@@ -592,9 +606,11 @@ public void testAlignmentWithPadding() throws Exception {
592606
FileSystem fs = path.getFileSystem(conf);
593607
long fileLen = fs.getFileStatus(path).getLen();
594608

595-
FSDataInputStream data = fs.open(path);
596-
data.seek(fileLen - 8); // 4-byte offset + "PAR1"
597-
long footerLen = BytesUtils.readIntLittleEndian(data);
609+
long footerLen;
610+
try (FSDataInputStream data = fs.open(path)) {
611+
data.seek(fileLen - 8); // 4-byte offset + "PAR1"
612+
footerLen = BytesUtils.readIntLittleEndian(data);
613+
}
598614
long startFooter = fileLen - footerLen - 8;
599615

600616
assertEquals("Footer should start after second row group without padding", secondRowGroupEnds, startFooter);
@@ -677,6 +693,8 @@ public void testAlignmentWithNoPaddingNeeded() throws Exception {
677693
Configuration conf = getTestConfiguration();
678694
// Disable writing out checksums as hardcoded byte offsets in assertions below expect it
679695
conf.setBoolean(ParquetOutputFormat.PAGE_WRITE_CHECKSUM_ENABLED, false);
696+
// close any filesystems to ensure that the the FS used by the writer picks up the configuration
697+
FileSystem.closeAll();
680698

681699
// uses the test constructor
682700
ParquetFileWriter w = new ParquetFileWriter(conf, SCHEMA, path, 100, 50);
@@ -716,9 +734,11 @@ public void testAlignmentWithNoPaddingNeeded() throws Exception {
716734
FileSystem fs = path.getFileSystem(conf);
717735
long fileLen = fs.getFileStatus(path).getLen();
718736

719-
FSDataInputStream data = fs.open(path);
720-
data.seek(fileLen - 8); // 4-byte offset + "PAR1"
721-
long footerLen = BytesUtils.readIntLittleEndian(data);
737+
long footerLen;
738+
try (FSDataInputStream data = fs.open(path)) {
739+
data.seek(fileLen - 8); // 4-byte offset + "PAR1"
740+
footerLen = BytesUtils.readIntLittleEndian(data);
741+
}
722742
long startFooter = fileLen - footerLen - 8;
723743

724744
assertEquals("Footer should start after second row group without padding", secondRowGroupEnds, startFooter);
@@ -975,6 +995,8 @@ public void testWriteReadStatisticsAllNulls() throws Exception {
975995
configuration.setBoolean("parquet.strings.signed-min-max.enabled", true);
976996
GroupWriteSupport.setSchema(schema, configuration);
977997

998+
// close any filesystems to ensure that the the FS used by the writer picks up the configuration
999+
FileSystem.closeAll();
9781000
ParquetWriter<Group> writer = new ParquetWriter<Group>(path, configuration, new GroupWriteSupport());
9791001

9801002
Group r1 = new SimpleGroup(schema);

pom.xml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -589,6 +589,7 @@
589589
<exclude>org.apache.parquet.conf.PlainParquetConfiguration#getClass(java.lang.String,java.lang.Class,java.lang.Class)</exclude>
590590
<exclude>org.apache.parquet.conf.ParquetConfiguration#getClass(java.lang.String,java.lang.Class,java.lang.Class)</exclude>
591591
<exclude>org.apache.parquet.hadoop.util.SerializationUtil#readObjectFromConfAsBase64(java.lang.String,org.apache.parquet.conf.ParquetConfiguration)</exclude>
592+
<exclude>org.apache.parquet.hadoop.util.vectorio.BindingUtils#awaitFuture(java.util.concurrent.Future,long,java.util.concurrent.TimeUnit)</exclude>
592593
<exclude>org.apache.parquet.conf.HadoopParquetConfiguration#getClass(java.lang.String,java.lang.Class,java.lang.Class)</exclude>
593594
<exclude>org.apache.parquet.avro.AvroParquetReader#builder(org.apache.parquet.io.InputFile,org.apache.parquet.conf.ParquetConfiguration)</exclude>
594595
<exclude>org.apache.parquet.hadoop.thrift.TBaseWriteSupport#setThriftClass(org.apache.parquet.conf.ParquetConfiguration,java.lang.Class)</exclude>

0 commit comments

Comments
 (0)