Skip to content

Commit 0d520b1

Browse files
committed
fixes #3122
1 parent 3306fd6 commit 0d520b1

File tree

8 files changed

+412
-38
lines changed

8 files changed

+412
-38
lines changed

parquet-hadoop/src/main/java/org/apache/parquet/crypto/AesCtrDecryptor.java

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,11 @@ public byte[] decrypt(byte[] lengthAndCiphertext, byte[] AAD) {
5555
public byte[] decrypt(byte[] ciphertext, int cipherTextOffset, int cipherTextLength, byte[] AAD) {
5656

5757
int plainTextLength = cipherTextLength - NONCE_LENGTH;
58-
if (plainTextLength < 1) {
58+
if (plainTextLength == 0) {
59+
return new byte[0];
60+
}
61+
62+
if (plainTextLength < 0) {
5963
throw new ParquetCryptoRuntimeException("Wrong input length " + plainTextLength);
6064
}
6165

@@ -91,7 +95,11 @@ public ByteBuffer decrypt(ByteBuffer ciphertext, byte[] AAD) {
9195
int cipherTextLength = ciphertext.limit() - ciphertext.position() - SIZE_LENGTH;
9296

9397
int plainTextLength = cipherTextLength - NONCE_LENGTH;
94-
if (plainTextLength < 1) {
98+
if (plainTextLength == 0) {
99+
return ByteBuffer.allocate(0);
100+
}
101+
102+
if (plainTextLength < 0) {
95103
throw new ParquetCryptoRuntimeException("Wrong input length " + plainTextLength);
96104
}
97105

parquet-hadoop/src/main/java/org/apache/parquet/crypto/AesGcmDecryptor.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,11 @@ public byte[] decrypt(byte[] lengthAndCiphertext, byte[] AAD) {
5151
public byte[] decrypt(byte[] ciphertext, int cipherTextOffset, int cipherTextLength, byte[] AAD) {
5252

5353
int plainTextLength = cipherTextLength - GCM_TAG_LENGTH - NONCE_LENGTH;
54-
if (plainTextLength < 1) {
54+
if (plainTextLength == 0) {
55+
return new byte[0];
56+
}
57+
58+
if (plainTextLength < 0) {
5559
throw new ParquetCryptoRuntimeException("Wrong input length " + plainTextLength);
5660
}
5761

@@ -81,7 +85,12 @@ public ByteBuffer decrypt(ByteBuffer ciphertext, byte[] AAD) {
8185
int cipherTextOffset = SIZE_LENGTH;
8286
int cipherTextLength = ciphertext.limit() - ciphertext.position() - SIZE_LENGTH;
8387
int plainTextLength = cipherTextLength - GCM_TAG_LENGTH - NONCE_LENGTH;
84-
if (plainTextLength < 1) {
88+
89+
if (plainTextLength == 0) {
90+
return ByteBuffer.allocate(0);
91+
}
92+
93+
if (plainTextLength < 0) {
8594
throw new ParquetCryptoRuntimeException("Wrong input length " + plainTextLength);
8695
}
8796

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

Lines changed: 109 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -1981,7 +1981,8 @@ public void writeDataPageV2Header(
19811981
rowCount,
19821982
dataEncoding,
19831983
rlByteLength,
1984-
dlByteLength),
1984+
dlByteLength,
1985+
true /* compressed by default */),
19851986
to);
19861987
}
19871988

@@ -2059,6 +2060,10 @@ public void writeDataPageV1Header(
20592060
pageHeaderAAD);
20602061
}
20612062

2063+
/**
2064+
* @deprecated will be removed in 2.0.0. Use {@link ParquetMetadataConverter#writeDataPageV2Header(int, int, int, int, int, org.apache.parquet.column.Encoding, int, int, boolean, OutputStream)} instead
2065+
*/
2066+
@Deprecated
20622067
public void writeDataPageV2Header(
20632068
int uncompressedSize,
20642069
int compressedSize,
@@ -2079,11 +2084,16 @@ public void writeDataPageV2Header(
20792084
dataEncoding,
20802085
rlByteLength,
20812086
dlByteLength,
2087+
true, /* compressed by default */
20822088
to,
20832089
null,
20842090
null);
20852091
}
20862092

2093+
/**
2094+
* @deprecated will be removed in 2.0.0. Use {@link ParquetMetadataConverter#writeDataPageV2Header(int, int, int, int, int, org.apache.parquet.column.Encoding, int, int, boolean, OutputStream, BlockCipher.Encryptor, byte[])} instead
2095+
*/
2096+
@Deprecated
20872097
public void writeDataPageV2Header(
20882098
int uncompressedSize,
20892099
int compressedSize,
@@ -2097,35 +2107,53 @@ public void writeDataPageV2Header(
20972107
BlockCipher.Encryptor blockEncryptor,
20982108
byte[] pageHeaderAAD)
20992109
throws IOException {
2100-
writePageHeader(
2101-
newDataPageV2Header(
2102-
uncompressedSize,
2103-
compressedSize,
2104-
valueCount,
2105-
nullCount,
2106-
rowCount,
2107-
dataEncoding,
2108-
rlByteLength,
2109-
dlByteLength),
2110+
writeDataPageV2Header(
2111+
uncompressedSize,
2112+
compressedSize,
2113+
valueCount,
2114+
nullCount,
2115+
rowCount,
2116+
dataEncoding,
2117+
rlByteLength,
2118+
dlByteLength,
2119+
true, /* compressed by default */
21102120
to,
21112121
blockEncryptor,
21122122
pageHeaderAAD);
21132123
}
21142124

2115-
private PageHeader newDataPageV2Header(
2125+
/**
2126+
* @deprecated will be removed in 2.0.0. Use {@link ParquetMetadataConverter#writeDataPageV2Header(int, int, int, int, int, org.apache.parquet.column.Encoding, int, int, boolean, int, OutputStream, BlockCipher.Encryptor, byte[])} instead
2127+
*/
2128+
@Deprecated
2129+
public void writeDataPageV2Header(
21162130
int uncompressedSize,
21172131
int compressedSize,
21182132
int valueCount,
21192133
int nullCount,
21202134
int rowCount,
21212135
org.apache.parquet.column.Encoding dataEncoding,
21222136
int rlByteLength,
2123-
int dlByteLength) {
2124-
DataPageHeaderV2 dataPageHeaderV2 = new DataPageHeaderV2(
2125-
valueCount, nullCount, rowCount, getEncoding(dataEncoding), dlByteLength, rlByteLength);
2126-
PageHeader pageHeader = new PageHeader(PageType.DATA_PAGE_V2, uncompressedSize, compressedSize);
2127-
pageHeader.setData_page_header_v2(dataPageHeaderV2);
2128-
return pageHeader;
2137+
int dlByteLength,
2138+
int crc,
2139+
OutputStream to,
2140+
BlockCipher.Encryptor blockEncryptor,
2141+
byte[] pageHeaderAAD)
2142+
throws IOException {
2143+
writeDataPageV2Header(
2144+
uncompressedSize,
2145+
compressedSize,
2146+
valueCount,
2147+
nullCount,
2148+
rowCount,
2149+
dataEncoding,
2150+
rlByteLength,
2151+
dlByteLength,
2152+
true, /* compressed by default */
2153+
crc,
2154+
to,
2155+
blockEncryptor,
2156+
pageHeaderAAD);
21292157
}
21302158

21312159
public void writeDataPageV2Header(
@@ -2137,7 +2165,34 @@ public void writeDataPageV2Header(
21372165
org.apache.parquet.column.Encoding dataEncoding,
21382166
int rlByteLength,
21392167
int dlByteLength,
2140-
int crc,
2168+
boolean compressed,
2169+
OutputStream to)
2170+
throws IOException {
2171+
writeDataPageV2Header(
2172+
uncompressedSize,
2173+
compressedSize,
2174+
valueCount,
2175+
nullCount,
2176+
rowCount,
2177+
dataEncoding,
2178+
rlByteLength,
2179+
dlByteLength,
2180+
compressed,
2181+
to,
2182+
null,
2183+
null);
2184+
}
2185+
2186+
public void writeDataPageV2Header(
2187+
int uncompressedSize,
2188+
int compressedSize,
2189+
int valueCount,
2190+
int nullCount,
2191+
int rowCount,
2192+
org.apache.parquet.column.Encoding dataEncoding,
2193+
int rlByteLength,
2194+
int dlByteLength,
2195+
boolean compressed,
21412196
OutputStream to,
21422197
BlockCipher.Encryptor blockEncryptor,
21432198
byte[] pageHeaderAAD)
@@ -2152,12 +2207,43 @@ public void writeDataPageV2Header(
21522207
dataEncoding,
21532208
rlByteLength,
21542209
dlByteLength,
2155-
crc),
2210+
compressed),
21562211
to,
21572212
blockEncryptor,
21582213
pageHeaderAAD);
21592214
}
21602215

2216+
public void writeDataPageV2Header(
2217+
int uncompressedSize,
2218+
int compressedSize,
2219+
int valueCount,
2220+
int nullCount,
2221+
int rowCount,
2222+
org.apache.parquet.column.Encoding dataEncoding,
2223+
int rlByteLength,
2224+
int dlByteLength,
2225+
boolean compressed,
2226+
int crc,
2227+
OutputStream to,
2228+
BlockCipher.Encryptor blockEncryptor,
2229+
byte[] pageHeaderAAD)
2230+
throws IOException {
2231+
PageHeader pageHeader = newDataPageV2Header(
2232+
uncompressedSize,
2233+
compressedSize,
2234+
valueCount,
2235+
nullCount,
2236+
rowCount,
2237+
dataEncoding,
2238+
rlByteLength,
2239+
dlByteLength,
2240+
compressed);
2241+
2242+
pageHeader.setCrc(crc);
2243+
2244+
writePageHeader(pageHeader, to, blockEncryptor, pageHeaderAAD);
2245+
}
2246+
21612247
private PageHeader newDataPageV2Header(
21622248
int uncompressedSize,
21632249
int compressedSize,
@@ -2167,12 +2253,13 @@ private PageHeader newDataPageV2Header(
21672253
org.apache.parquet.column.Encoding dataEncoding,
21682254
int rlByteLength,
21692255
int dlByteLength,
2170-
int crc) {
2256+
boolean compressed) {
21712257
DataPageHeaderV2 dataPageHeaderV2 = new DataPageHeaderV2(
21722258
valueCount, nullCount, rowCount, getEncoding(dataEncoding), dlByteLength, rlByteLength);
2259+
dataPageHeaderV2.setIs_compressed(compressed);
2260+
21732261
PageHeader pageHeader = new PageHeader(PageType.DATA_PAGE_V2, uncompressedSize, compressedSize);
21742262
pageHeader.setData_page_header_v2(dataPageHeaderV2);
2175-
pageHeader.setCrc(crc);
21762263
return pageHeader;
21772264
}
21782265

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

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -295,8 +295,13 @@ 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);
298+
boolean compressed = false;
299+
BytesInput compressedData = BytesInput.empty();
300+
if (data.size() > 0) {
301+
// TODO: decide if we compress
302+
compressedData = compressor.compress(data);
303+
compressed = true;
304+
}
300305
if (null != pageBlockEncryptor) {
301306
AesCipher.quickUpdatePageAAD(dataPageAAD, pageOrdinal);
302307
compressedData = BytesInput.from(pageBlockEncryptor.encrypt(compressedData.toByteArray(), dataPageAAD));
@@ -327,6 +332,7 @@ public void writePageV2(
327332
dataEncoding,
328333
rlByteLength,
329334
dlByteLength,
335+
compressed,
330336
(int) crc.getValue(),
331337
tempOutputStream,
332338
headerBlockEncryptor,
@@ -341,6 +347,7 @@ public void writePageV2(
341347
dataEncoding,
342348
rlByteLength,
343349
dlByteLength,
350+
compressed,
344351
tempOutputStream,
345352
headerBlockEncryptor,
346353
dataPageHeaderAAD);

0 commit comments

Comments
 (0)