/
ParquetReader.java
273 lines (244 loc) · 11.4 KB
/
ParquetReader.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.hive.parquet.reader;
import com.facebook.presto.hive.parquet.Field;
import com.facebook.presto.hive.parquet.GroupField;
import com.facebook.presto.hive.parquet.ParquetCorruptionException;
import com.facebook.presto.hive.parquet.ParquetDataSource;
import com.facebook.presto.hive.parquet.PrimitiveField;
import com.facebook.presto.hive.parquet.RichColumnDescriptor;
import com.facebook.presto.memory.context.AggregatedMemoryContext;
import com.facebook.presto.memory.context.LocalMemoryContext;
import com.facebook.presto.spi.block.ArrayBlock;
import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.RowBlock;
import com.facebook.presto.spi.block.RunLengthEncodedBlock;
import com.facebook.presto.spi.type.MapType;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeSignatureParameter;
import it.unimi.dsi.fastutil.booleans.BooleanArrayList;
import it.unimi.dsi.fastutil.booleans.BooleanList;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import parquet.column.ColumnDescriptor;
import parquet.hadoop.metadata.BlockMetaData;
import parquet.hadoop.metadata.ColumnChunkMetaData;
import parquet.hadoop.metadata.ColumnPath;
import parquet.io.MessageColumnIO;
import parquet.io.PrimitiveColumnIO;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import static com.facebook.presto.hive.parquet.ParquetValidationUtils.validateParquet;
import static com.facebook.presto.hive.parquet.reader.ParquetListColumnReader.calculateCollectionOffsets;
import static com.facebook.presto.spi.type.StandardTypes.ARRAY;
import static com.facebook.presto.spi.type.StandardTypes.MAP;
import static com.facebook.presto.spi.type.StandardTypes.ROW;
import static com.google.common.base.Preconditions.checkArgument;
import static java.lang.Math.min;
import static java.lang.Math.toIntExact;
import static java.util.Objects.requireNonNull;
public class ParquetReader
implements Closeable
{
private static final int MAX_VECTOR_LENGTH = 1024;
private final List<BlockMetaData> blocks;
private final List<PrimitiveColumnIO> columns;
private final ParquetDataSource dataSource;
private final AggregatedMemoryContext systemMemoryContext;
private int currentBlock;
private BlockMetaData currentBlockMetadata;
private long currentPosition;
private long currentGroupRowCount;
private long nextRowInGroup;
private int batchSize;
private final ParquetPrimitiveColumnReader[] columnReaders;
private AggregatedMemoryContext currentRowGroupMemoryContext;
public ParquetReader(MessageColumnIO messageColumnIO,
List<BlockMetaData> blocks,
ParquetDataSource dataSource,
AggregatedMemoryContext systemMemoryContext)
{
this.blocks = blocks;
this.dataSource = requireNonNull(dataSource, "dataSource is null");
this.systemMemoryContext = requireNonNull(systemMemoryContext, "systemMemoryContext is null");
this.currentRowGroupMemoryContext = systemMemoryContext.newAggregatedMemoryContext();
columns = messageColumnIO.getLeaves();
columnReaders = new ParquetPrimitiveColumnReader[columns.size()];
}
@Override
public void close()
throws IOException
{
currentRowGroupMemoryContext.close();
dataSource.close();
}
public long getPosition()
{
return currentPosition;
}
public int nextBatch()
{
if (nextRowInGroup >= currentGroupRowCount && !advanceToNextRowGroup()) {
return -1;
}
batchSize = toIntExact(min(MAX_VECTOR_LENGTH, currentGroupRowCount - nextRowInGroup));
nextRowInGroup += batchSize;
currentPosition += batchSize;
Arrays.stream(columnReaders)
.forEach(reader -> reader.prepareNextRead(batchSize));
return batchSize;
}
private boolean advanceToNextRowGroup()
{
currentRowGroupMemoryContext.close();
currentRowGroupMemoryContext = systemMemoryContext.newAggregatedMemoryContext();
if (currentBlock == blocks.size()) {
return false;
}
currentBlockMetadata = blocks.get(currentBlock);
currentBlock = currentBlock + 1;
nextRowInGroup = 0L;
currentGroupRowCount = currentBlockMetadata.getRowCount();
initializeColumnReaders();
return true;
}
private ColumnChunk readArray(GroupField field)
throws IOException
{
List<Type> parameters = field.getType().getTypeParameters();
checkArgument(parameters.size() == 1, "Arrays must have a single type parameter, found %d", parameters.size());
Field elementField = field.getChildren().get(0).get();
ColumnChunk columnChunk = readColumnChunk(elementField);
IntList offsets = new IntArrayList();
BooleanList valueIsNull = new BooleanArrayList();
calculateCollectionOffsets(field, offsets, valueIsNull, columnChunk.getDefinitionLevels(), columnChunk.getRepetitionLevels());
Block arrayBlock = ArrayBlock.fromElementBlock(valueIsNull.size(), Optional.of(valueIsNull.toBooleanArray()), offsets.toIntArray(), columnChunk.getBlock());
return new ColumnChunk(arrayBlock, columnChunk.getDefinitionLevels(), columnChunk.getRepetitionLevels());
}
private ColumnChunk readMap(GroupField field)
throws IOException
{
List<Type> parameters = field.getType().getTypeParameters();
checkArgument(parameters.size() == 2, "Maps must have two type parameters, found %d", parameters.size());
Block[] blocks = new Block[parameters.size()];
ColumnChunk columnChunk = readColumnChunk(field.getChildren().get(0).get());
blocks[0] = columnChunk.getBlock();
blocks[1] = readColumnChunk(field.getChildren().get(1).get()).getBlock();
IntList offsets = new IntArrayList();
BooleanList valueIsNull = new BooleanArrayList();
calculateCollectionOffsets(field, offsets, valueIsNull, columnChunk.getDefinitionLevels(), columnChunk.getRepetitionLevels());
Block mapBlock = ((MapType) field.getType()).createBlockFromKeyValue(Optional.of(valueIsNull.toBooleanArray()), offsets.toIntArray(), blocks[0], blocks[1]);
return new ColumnChunk(mapBlock, columnChunk.getDefinitionLevels(), columnChunk.getRepetitionLevels());
}
private ColumnChunk readStruct(GroupField field)
throws IOException
{
List<TypeSignatureParameter> fields = field.getType().getTypeSignature().getParameters();
Block[] blocks = new Block[fields.size()];
ColumnChunk columnChunk = null;
List<Optional<Field>> parameters = field.getChildren();
for (int i = 0; i < fields.size(); i++) {
Optional<Field> parameter = parameters.get(i);
if (parameter.isPresent()) {
columnChunk = readColumnChunk(parameter.get());
blocks[i] = columnChunk.getBlock();
}
}
for (int i = 0; i < fields.size(); i++) {
if (blocks[i] == null) {
blocks[i] = RunLengthEncodedBlock.create(field.getType(), null, columnChunk.getBlock().getPositionCount());
}
}
BooleanList structIsNull = ParquetStructColumnReader.calculateStructOffsets(field, columnChunk.getDefinitionLevels(), columnChunk.getRepetitionLevels());
boolean[] structIsNullVector = structIsNull.toBooleanArray();
Block rowBlock = RowBlock.fromFieldBlocks(structIsNullVector.length, Optional.of(structIsNullVector), blocks);
return new ColumnChunk(rowBlock, columnChunk.getDefinitionLevels(), columnChunk.getRepetitionLevels());
}
private ColumnChunk readPrimitive(PrimitiveField field)
throws IOException
{
ColumnDescriptor columnDescriptor = field.getDescriptor();
ParquetPrimitiveColumnReader columnReader = columnReaders[field.getId()];
if (columnReader.getPageReader() == null) {
validateParquet(currentBlockMetadata.getRowCount() > 0, "Row group has 0 rows");
ColumnChunkMetaData metadata = getColumnChunkMetaData(columnDescriptor);
long startingPosition = metadata.getStartingPos();
int totalSize = toIntExact(metadata.getTotalSize());
byte[] buffer = allocateBlock(totalSize);
dataSource.readFully(startingPosition, buffer);
ParquetColumnChunkDescriptor descriptor = new ParquetColumnChunkDescriptor(columnDescriptor, metadata, totalSize);
ParquetColumnChunk columnChunk = new ParquetColumnChunk(descriptor, buffer, 0);
columnReader.setPageReader(columnChunk.readAllPages());
}
return columnReader.readPrimitive(field);
}
private byte[] allocateBlock(int length)
{
byte[] buffer = new byte[length];
LocalMemoryContext blockMemoryContext = currentRowGroupMemoryContext.newLocalMemoryContext(ParquetReader.class.getSimpleName());
blockMemoryContext.setBytes(buffer.length);
return buffer;
}
private ColumnChunkMetaData getColumnChunkMetaData(ColumnDescriptor columnDescriptor)
throws IOException
{
for (ColumnChunkMetaData metadata : currentBlockMetadata.getColumns()) {
if (metadata.getPath().equals(ColumnPath.get(columnDescriptor.getPath()))) {
return metadata;
}
}
throw new ParquetCorruptionException("Metadata is missing for column: %s", columnDescriptor);
}
private void initializeColumnReaders()
{
for (PrimitiveColumnIO columnIO : columns) {
RichColumnDescriptor column = new RichColumnDescriptor(columnIO.getColumnDescriptor(), columnIO.getType().asPrimitiveType());
columnReaders[columnIO.getId()] = ParquetPrimitiveColumnReader.createReader(column);
}
}
public Block readBlock(Field field)
throws IOException
{
return readColumnChunk(field).getBlock();
}
private ColumnChunk readColumnChunk(Field field)
throws IOException
{
ColumnChunk columnChunk;
if (ROW.equals(field.getType().getTypeSignature().getBase())) {
columnChunk = readStruct((GroupField) field);
}
else if (MAP.equals(field.getType().getTypeSignature().getBase())) {
columnChunk = readMap((GroupField) field);
}
else if (ARRAY.equals(field.getType().getTypeSignature().getBase())) {
columnChunk = readArray((GroupField) field);
}
else {
columnChunk = readPrimitive((PrimitiveField) field);
}
return columnChunk;
}
public ParquetDataSource getDataSource()
{
return dataSource;
}
public AggregatedMemoryContext getSystemMemoryContext()
{
return systemMemoryContext;
}
}