|
27 | 27 | import java.nio.ByteBuffer; |
28 | 28 | import java.util.ArrayList; |
29 | 29 | import java.util.Arrays; |
| 30 | +import java.util.Collection; |
30 | 31 | import java.util.HashMap; |
31 | 32 | import java.util.List; |
32 | 33 | import java.util.Map; |
| 34 | +import java.util.Set; |
33 | 35 | import java.util.concurrent.ExecutionException; |
34 | 36 | import java.util.concurrent.ExecutorService; |
35 | 37 | import java.util.concurrent.Future; |
|
40 | 42 | import org.slf4j.Logger; |
41 | 43 | import org.slf4j.LoggerFactory; |
42 | 44 |
|
| 45 | +import org.apache.hadoop.conf.Configuration; |
| 46 | +import org.apache.hadoop.fs.Path; |
| 47 | +import org.apache.parquet.HadoopReadOptions; |
43 | 48 | import org.apache.parquet.ParquetReadOptions; |
44 | 49 | import org.apache.parquet.Preconditions; |
45 | 50 | import org.apache.parquet.bytes.ByteBufferInputStream; |
|
53 | 58 | import org.apache.parquet.column.page.PageReadStore; |
54 | 59 | import org.apache.parquet.compression.CompressionCodecFactory; |
55 | 60 | import org.apache.parquet.crypto.AesCipher; |
| 61 | +import org.apache.parquet.crypto.EncryptionPropertiesFactory; |
56 | 62 | import org.apache.parquet.crypto.FileDecryptionProperties; |
57 | 63 | import org.apache.parquet.crypto.InternalColumnDecryptionSetup; |
58 | 64 | import org.apache.parquet.crypto.InternalFileDecryptor; |
|
67 | 73 | import org.apache.parquet.format.PageHeader; |
68 | 74 | import org.apache.parquet.format.Util; |
69 | 75 | import org.apache.parquet.format.converter.ParquetMetadataConverter; |
| 76 | +import org.apache.parquet.hadoop.ParquetInputFormat; |
70 | 77 | import org.apache.parquet.hadoop.metadata.BlockMetaData; |
71 | 78 | import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; |
72 | 79 | import org.apache.parquet.hadoop.metadata.ColumnPath; |
@@ -128,6 +135,48 @@ public FileReader(InputFile file, ParquetReadOptions options, ReadOptions cometO |
128 | 135 | this(file, null, options, cometOptions, null); |
129 | 136 | } |
130 | 137 |
|
| 138 | + /** This constructor is called from Apache Iceberg. */ |
| 139 | + public FileReader( |
| 140 | + Path path, |
| 141 | + Configuration conf, |
| 142 | + ReadOptions cometOptions, |
| 143 | + Map<String, String> properties, |
| 144 | + Long start, |
| 145 | + Long length, |
| 146 | + byte[] fileEncryptionKey, |
| 147 | + byte[] fileAADPrefix) |
| 148 | + throws IOException { |
| 149 | + ParquetReadOptions options = |
| 150 | + buildParquetReadOptions(conf, properties, start, length, fileEncryptionKey, fileAADPrefix); |
| 151 | + this.converter = new ParquetMetadataConverter(options); |
| 152 | + this.file = CometInputFile.fromPath(path, conf); |
| 153 | + this.f = file.newStream(); |
| 154 | + this.options = options; |
| 155 | + this.cometOptions = cometOptions; |
| 156 | + this.metrics = null; |
| 157 | + try { |
| 158 | + this.footer = readFooter(file, options, f, converter); |
| 159 | + } catch (Exception e) { |
| 160 | + // In case that reading footer throws an exception in the constructor, the new stream |
| 161 | + // should be closed. Otherwise, there's no way to close this outside. |
| 162 | + f.close(); |
| 163 | + throw e; |
| 164 | + } |
| 165 | + this.fileMetaData = footer.getFileMetaData(); |
| 166 | + this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called before filterRowGroups! |
| 167 | + if (null != fileDecryptor && fileDecryptor.plaintextFile()) { |
| 168 | + this.fileDecryptor = null; // Plaintext file. No need in decryptor |
| 169 | + } |
| 170 | + |
| 171 | + this.blocks = footer.getBlocks(); // filter row group in iceberg |
| 172 | + this.blockIndexStores = listWithNulls(this.blocks.size()); |
| 173 | + this.blockRowRanges = listWithNulls(this.blocks.size()); |
| 174 | + for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) { |
| 175 | + paths.put(ColumnPath.get(col.getPath()), col); |
| 176 | + } |
| 177 | + this.crc = options.usePageChecksumVerification() ? new CRC32() : null; |
| 178 | + } |
| 179 | + |
131 | 180 | public FileReader( |
132 | 181 | InputFile file, |
133 | 182 | ParquetReadOptions options, |
@@ -209,6 +258,57 @@ public void setRequestedSchema(List<ColumnDescriptor> projection) { |
209 | 258 | } |
210 | 259 | } |
211 | 260 |
|
| 261 | + /** This method is called from Apache Iceberg. */ |
| 262 | + public void setRequestedSchemaFromSpecs(List<ParquetColumnSpec> specList) { |
| 263 | + paths.clear(); |
| 264 | + for (ParquetColumnSpec colSpec : specList) { |
| 265 | + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(colSpec); |
| 266 | + paths.put(ColumnPath.get(colSpec.getPath()), descriptor); |
| 267 | + } |
| 268 | + } |
| 269 | + |
| 270 | + private static ParquetReadOptions buildParquetReadOptions( |
| 271 | + Configuration conf, |
| 272 | + Map<String, String> properties, |
| 273 | + Long start, |
| 274 | + Long length, |
| 275 | + byte[] fileEncryptionKey, |
| 276 | + byte[] fileAADPrefix) { |
| 277 | + |
| 278 | + // Iceberg remove these read properties when building the ParquetReadOptions. |
| 279 | + // We want build the exact same ParquetReadOptions as Iceberg's. |
| 280 | + Collection<String> readPropertiesToRemove = |
| 281 | + Set.of( |
| 282 | + ParquetInputFormat.UNBOUND_RECORD_FILTER, |
| 283 | + ParquetInputFormat.FILTER_PREDICATE, |
| 284 | + ParquetInputFormat.READ_SUPPORT_CLASS, |
| 285 | + EncryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME); |
| 286 | + |
| 287 | + for (String property : readPropertiesToRemove) { |
| 288 | + conf.unset(property); |
| 289 | + } |
| 290 | + |
| 291 | + ParquetReadOptions.Builder optionsBuilder = HadoopReadOptions.builder(conf); |
| 292 | + for (Map.Entry<String, String> entry : properties.entrySet()) { |
| 293 | + optionsBuilder.set(entry.getKey(), entry.getValue()); |
| 294 | + } |
| 295 | + |
| 296 | + if (start != null && length != null) { |
| 297 | + optionsBuilder.withRange(start, start + length); |
| 298 | + } |
| 299 | + |
| 300 | + if (fileEncryptionKey != null) { |
| 301 | + FileDecryptionProperties fileDecryptionProperties = |
| 302 | + FileDecryptionProperties.builder() |
| 303 | + .withFooterKey(fileEncryptionKey) |
| 304 | + .withAADPrefix(fileAADPrefix) |
| 305 | + .build(); |
| 306 | + optionsBuilder.withDecryption(fileDecryptionProperties); |
| 307 | + } |
| 308 | + |
| 309 | + return optionsBuilder.build(); |
| 310 | + } |
| 311 | + |
212 | 312 | /** |
213 | 313 | * Gets the total number of records across all row groups (after applying row group filtering). |
214 | 314 | */ |
@@ -245,15 +345,15 @@ public boolean skipNextRowGroup() { |
245 | 345 | * Returns the next row group to read (after applying row group filtering), or null if there's no |
246 | 346 | * more row group. |
247 | 347 | */ |
248 | | - public PageReadStore readNextRowGroup() throws IOException { |
| 348 | + public RowGroupReader readNextRowGroup() throws IOException { |
249 | 349 | if (currentBlock == blocks.size()) { |
250 | 350 | return null; |
251 | 351 | } |
252 | 352 | BlockMetaData block = blocks.get(currentBlock); |
253 | 353 | if (block.getRowCount() == 0) { |
254 | 354 | throw new RuntimeException("Illegal row group of 0 rows"); |
255 | 355 | } |
256 | | - this.currentRowGroup = new RowGroupReader(block.getRowCount()); |
| 356 | + this.currentRowGroup = new RowGroupReader(block.getRowCount(), block.getRowIndexOffset()); |
257 | 357 | // prepare the list of consecutive parts to read them in one scan |
258 | 358 | List<ConsecutivePartList> allParts = new ArrayList<>(); |
259 | 359 | ConsecutivePartList currentParts = null; |
@@ -362,7 +462,7 @@ ColumnIndexReader getColumnIndexReader(int blockIndex) { |
362 | 462 | return ciStore; |
363 | 463 | } |
364 | 464 |
|
365 | | - private PageReadStore readChunks( |
| 465 | + private RowGroupReader readChunks( |
366 | 466 | BlockMetaData block, List<ConsecutivePartList> allParts, ChunkListBuilder builder) |
367 | 467 | throws IOException { |
368 | 468 | if (shouldReadParallel()) { |
|
0 commit comments