Skip to content

Commit 08b7936

Browse files
committed
fixes #3122
1 parent 3306fd6 commit 08b7936

File tree

9 files changed

+169
-45
lines changed

9 files changed

+169
-45
lines changed

parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV2.java

Lines changed: 41 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,7 @@ public static DataPageV2 uncompressed(
5353
definitionLevels,
5454
dataEncoding,
5555
data,
56+
0,
5657
Math.toIntExact(repetitionLevels.size() + definitionLevels.size() + data.size()),
5758
statistics,
5859
false);
@@ -89,6 +90,7 @@ public static DataPageV2 uncompressed(
8990
definitionLevels,
9091
dataEncoding,
9192
data,
93+
0,
9294
Math.toIntExact(repetitionLevels.size() + definitionLevels.size() + data.size()),
9395
statistics,
9496
false);
@@ -124,6 +126,7 @@ public static DataPageV2 compressed(
124126
definitionLevels,
125127
dataEncoding,
126128
data,
129+
Math.toIntExact(repetitionLevels.size() + definitionLevels.size() + data.size()),
127130
uncompressedSize,
128131
statistics,
129132
true);
@@ -138,6 +141,10 @@ public static DataPageV2 compressed(
138141
private final Statistics<?> statistics;
139142
private final boolean isCompressed;
140143

144+
/**
145+
* @deprecated will be removed in 2.0.0. Use {@link DataPageV2#DataPageV2(int, int, int, long, BytesInput, BytesInput, Encoding, BytesInput, int, int, Statistics, boolean)} instead
146+
*/
147+
@Deprecated
141148
public DataPageV2(
142149
int rowCount,
143150
int nullCount,
@@ -163,6 +170,33 @@ public DataPageV2(
163170
this.isCompressed = isCompressed;
164171
}
165172

173+
public DataPageV2(
174+
int rowCount,
175+
int nullCount,
176+
int valueCount,
177+
BytesInput repetitionLevels,
178+
BytesInput definitionLevels,
179+
Encoding dataEncoding,
180+
BytesInput data,
181+
int compressedSize,
182+
int uncompressedSize,
183+
Statistics<?> statistics,
184+
boolean isCompressed) {
185+
super(compressedSize, uncompressedSize, valueCount);
186+
if (!isCompressed && compressedSize != 0) {
187+
throw new IllegalArgumentException("compressedSize must be 0 if page is not compressed");
188+
}
189+
190+
this.rowCount = rowCount;
191+
this.nullCount = nullCount;
192+
this.repetitionLevels = repetitionLevels;
193+
this.definitionLevels = definitionLevels;
194+
this.dataEncoding = dataEncoding;
195+
this.data = data;
196+
this.statistics = statistics;
197+
this.isCompressed = isCompressed;
198+
}
199+
166200
private DataPageV2(
167201
int rowCount,
168202
int nullCount,
@@ -172,14 +206,11 @@ private DataPageV2(
172206
BytesInput definitionLevels,
173207
Encoding dataEncoding,
174208
BytesInput data,
209+
int compressedSize,
175210
int uncompressedSize,
176211
Statistics<?> statistics,
177212
boolean isCompressed) {
178-
super(
179-
Math.toIntExact(repetitionLevels.size() + definitionLevels.size() + data.size()),
180-
uncompressedSize,
181-
valueCount,
182-
firstRowIndex);
213+
super(compressedSize, uncompressedSize, valueCount, firstRowIndex);
183214
this.rowCount = rowCount;
184215
this.nullCount = nullCount;
185216
this.repetitionLevels = repetitionLevels;
@@ -190,6 +221,11 @@ private DataPageV2(
190221
this.isCompressed = isCompressed;
191222
}
192223

224+
@Override
225+
public int getCompressedSize() {
226+
return isCompressed ? super.getCompressedSize() : 0;
227+
}
228+
193229
public int getRowCount() {
194230
return rowCount;
195231
}

parquet-column/src/main/java/org/apache/parquet/column/page/Page.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,9 @@ public abstract class Page {
3434
this.uncompressedSize = uncompressedSize;
3535
}
3636

37+
/**
38+
* @return the compressed size of the page when the bytes are compressed, otherwise return 0
39+
*/
3740
public int getCompressedSize() {
3841
return compressedSize;
3942
}

parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.parquet.column.impl;
2020

2121
import static junit.framework.Assert.assertEquals;
22+
import static junit.framework.Assert.assertFalse;
2223
import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_2_0;
2324

2425
import java.util.List;
@@ -135,4 +136,46 @@ public void testOptional() throws Exception {
135136
}
136137
assertEquals(0, converter.count);
137138
}
139+
140+
@Test
141+
public void testV2AllNullValues() throws Exception {
142+
MessageType schema = MessageTypeParser.parseMessageType("message test { optional binary foo; }");
143+
ColumnDescriptor col = schema.getColumns().get(0);
144+
MemPageWriter pageWriter = new MemPageWriter();
145+
ColumnWriterV2 columnWriterV2 = new ColumnWriterV2(
146+
col,
147+
pageWriter,
148+
ParquetProperties.builder()
149+
.withDictionaryPageSize(1024)
150+
.withWriterVersion(PARQUET_2_0)
151+
.withPageSize(2048)
152+
.build());
153+
for (int i = 0; i < rows; i++) {
154+
columnWriterV2.writeNull(0, 0);
155+
}
156+
columnWriterV2.writePage();
157+
columnWriterV2.finalizeColumnChunk();
158+
List<DataPage> pages = pageWriter.getPages();
159+
int valueCount = 0;
160+
int rowCount = 0;
161+
for (DataPage dataPage : pages) {
162+
DataPageV2 page = (DataPageV2) dataPage;
163+
valueCount += page.getValueCount();
164+
rowCount += page.getRowCount();
165+
assertFalse(page.isCompressed());
166+
assertEquals(0, page.getCompressedSize());
167+
}
168+
assertEquals(rows, rowCount);
169+
assertEquals(rows, valueCount);
170+
MemPageReader pageReader = new MemPageReader(rows, pages.iterator(), pageWriter.getDictionaryPage());
171+
ValidatingConverter converter = new ValidatingConverter();
172+
ColumnReader columnReader =
173+
new ColumnReaderImpl(col, pageReader, converter, VersionParser.parse(Version.FULL_VERSION));
174+
for (int i = 0; i < rows; i++) {
175+
assertEquals(0, columnReader.getCurrentRepetitionLevel());
176+
assertEquals(0, columnReader.getCurrentDefinitionLevel());
177+
columnReader.consume();
178+
}
179+
assertEquals(0, converter.count);
180+
}
138181
}

parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java

Lines changed: 14 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -2123,6 +2123,9 @@ private PageHeader newDataPageV2Header(
21232123
int dlByteLength) {
21242124
DataPageHeaderV2 dataPageHeaderV2 = new DataPageHeaderV2(
21252125
valueCount, nullCount, rowCount, getEncoding(dataEncoding), dlByteLength, rlByteLength);
2126+
if (compressedSize == 0) {
2127+
dataPageHeaderV2.setIs_compressed(false);
2128+
}
21262129
PageHeader pageHeader = new PageHeader(PageType.DATA_PAGE_V2, uncompressedSize, compressedSize);
21272130
pageHeader.setData_page_header_v2(dataPageHeaderV2);
21282131
return pageHeader;
@@ -2142,38 +2145,18 @@ public void writeDataPageV2Header(
21422145
BlockCipher.Encryptor blockEncryptor,
21432146
byte[] pageHeaderAAD)
21442147
throws IOException {
2145-
writePageHeader(
2146-
newDataPageV2Header(
2147-
uncompressedSize,
2148-
compressedSize,
2149-
valueCount,
2150-
nullCount,
2151-
rowCount,
2152-
dataEncoding,
2153-
rlByteLength,
2154-
dlByteLength,
2155-
crc),
2156-
to,
2157-
blockEncryptor,
2158-
pageHeaderAAD);
2159-
}
2160-
2161-
private PageHeader newDataPageV2Header(
2162-
int uncompressedSize,
2163-
int compressedSize,
2164-
int valueCount,
2165-
int nullCount,
2166-
int rowCount,
2167-
org.apache.parquet.column.Encoding dataEncoding,
2168-
int rlByteLength,
2169-
int dlByteLength,
2170-
int crc) {
2171-
DataPageHeaderV2 dataPageHeaderV2 = new DataPageHeaderV2(
2172-
valueCount, nullCount, rowCount, getEncoding(dataEncoding), dlByteLength, rlByteLength);
2173-
PageHeader pageHeader = new PageHeader(PageType.DATA_PAGE_V2, uncompressedSize, compressedSize);
2174-
pageHeader.setData_page_header_v2(dataPageHeaderV2);
2148+
PageHeader pageHeader = newDataPageV2Header(
2149+
uncompressedSize,
2150+
compressedSize,
2151+
valueCount,
2152+
nullCount,
2153+
rowCount,
2154+
dataEncoding,
2155+
rlByteLength,
2156+
dlByteLength);
21752157
pageHeader.setCrc(crc);
2176-
return pageHeader;
2158+
2159+
writePageHeader(pageHeader, to, blockEncryptor, pageHeaderAAD);
21772160
}
21782161

21792162
public void writeDictionaryPageHeader(

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

Lines changed: 12 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -295,14 +295,19 @@ public void writePageV2(
295295
int rlByteLength = toIntWithCheck(repetitionLevels.size());
296296
int dlByteLength = toIntWithCheck(definitionLevels.size());
297297
int uncompressedSize = toIntWithCheck(data.size() + repetitionLevels.size() + definitionLevels.size());
298-
// TODO: decide if we compress
299-
BytesInput compressedData = compressor.compress(data);
300-
if (null != pageBlockEncryptor) {
301-
AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
302-
compressedData = BytesInput.from(pageBlockEncryptor.encrypt(compressedData.toByteArray(), dataPageAAD));
298+
BytesInput compressedData = BytesInput.empty();
299+
int compressedSize = 0;
300+
if (data.size() > 0) {
301+
// TODO: decide if we compress
302+
compressedData = compressor.compress(data);
303+
if (null != pageBlockEncryptor) {
304+
AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
305+
compressedData =
306+
BytesInput.from(pageBlockEncryptor.encrypt(compressedData.toByteArray(), dataPageAAD));
307+
}
308+
compressedSize =
309+
toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size());
303310
}
304-
int compressedSize =
305-
toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size());
306311
tempOutputStream.reset();
307312
if (null != headerBlockEncryptor) {
308313
AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal);

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1969,6 +1969,7 @@ public ColumnChunkPageReader readAllPages(
19691969
definitionLevels,
19701970
converter.getEncoding(dataHeaderV2.getEncoding()),
19711971
values,
1972+
dataHeaderV2.isIs_compressed() ? compressedPageSize : 0,
19721973
uncompressedPageSize,
19731974
converter.fromParquetStatistics(
19741975
getFileMetaData().getCreatedBy(), dataHeaderV2.getStatistics(), type),

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

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1150,8 +1150,11 @@ public void writeDataPageV2(
11501150
int rlByteLength = toIntWithCheck(repetitionLevels.size(), "page repetition levels");
11511151
int dlByteLength = toIntWithCheck(definitionLevels.size(), "page definition levels");
11521152

1153-
int compressedSize =
1154-
toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size(), "page");
1153+
int compressedSize = 0;
1154+
if (compressedData.size() > 0) {
1155+
compressedSize =
1156+
toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size(), "page");
1157+
}
11551158

11561159
int uncompressedSize =
11571160
toIntWithCheck(uncompressedDataSize + repetitionLevels.size() + definitionLevels.size(), "page");

parquet-hadoop/src/test/java/org/apache/parquet/encodings/FileEncodingsIT.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -452,6 +452,7 @@ public DataPage visit(DataPageV2 data) {
452452
BytesInput.from(data.getDefinitionLevels().toByteArray()),
453453
data.getDataEncoding(),
454454
BytesInput.from(data.getData().toByteArray()),
455+
data.isCompressed() ? data.getCompressedSize() : 0,
455456
data.getUncompressedSize(),
456457
data.getStatistics(),
457458
data.isCompressed());

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

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,9 +33,11 @@
3333
import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
3434
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
3535
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
36+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
3637
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
3738
import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
3839
import static org.junit.Assert.assertEquals;
40+
import static org.junit.Assert.assertFalse;
3941
import static org.junit.Assert.assertNull;
4042
import static org.junit.Assert.assertTrue;
4143

@@ -60,6 +62,8 @@
6062
import org.apache.parquet.example.data.Group;
6163
import org.apache.parquet.example.data.GroupFactory;
6264
import org.apache.parquet.example.data.simple.SimpleGroupFactory;
65+
import org.apache.parquet.format.PageHeader;
66+
import org.apache.parquet.format.Util;
6367
import org.apache.parquet.hadoop.example.ExampleParquetWriter;
6468
import org.apache.parquet.hadoop.example.GroupReadSupport;
6569
import org.apache.parquet.hadoop.example.GroupWriteSupport;
@@ -592,4 +596,49 @@ public void testSizeStatisticsControl() throws Exception {
592596
}
593597
}
594598
}
599+
600+
@Test
601+
public void testV2WriteAllNullValues() throws Exception {
602+
MessageType schema = Types.buildMessage().optional(FLOAT).named("float").named("msg");
603+
604+
Configuration conf = new Configuration();
605+
GroupWriteSupport.setSchema(schema, conf);
606+
607+
File file = temp.newFile();
608+
temp.delete();
609+
Path path = new Path(file.getAbsolutePath());
610+
611+
SimpleGroupFactory factory = new SimpleGroupFactory(schema);
612+
Group nullValue = factory.newGroup();
613+
int recordCount = 10;
614+
615+
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(path)
616+
.withAllocator(allocator)
617+
.withConf(conf)
618+
.withWriterVersion(WriterVersion.PARQUET_2_0)
619+
.withDictionaryEncoding(false)
620+
.build()) {
621+
for (int i = 0; i < recordCount; i++) {
622+
writer.write(nullValue);
623+
}
624+
}
625+
626+
try (ParquetReader<Group> reader =
627+
ParquetReader.builder(new GroupReadSupport(), path).build()) {
628+
int readRecordCount = 0;
629+
for (Group group = reader.read(); group != null; group = reader.read()) {
630+
assertEquals(nullValue.toString(), group.toString());
631+
++readRecordCount;
632+
}
633+
assertEquals("Number of written records should be equal to the read one", recordCount, readRecordCount);
634+
}
635+
636+
try (ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(path, conf))) {
637+
BlockMetaData blockMetaData = reader.getFooter().getBlocks().get(0);
638+
reader.f.seek(blockMetaData.getStartingPos());
639+
PageHeader pageHeader = Util.readPageHeader(reader.f);
640+
assertFalse(pageHeader.getData_page_header_v2().isIs_compressed());
641+
assertEquals(0, pageHeader.getCompressed_page_size());
642+
}
643+
}
595644
}

0 commit comments

Comments
 (0)