-
Notifications
You must be signed in to change notification settings - Fork 4.6k
/
HiveContextAwareRecordReader.java
451 lines (400 loc) · 16.4 KB
/
HiveContextAwareRecordReader.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
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.hadoop.hive.ql.io;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger;
import org.apache.hadoop.mapred.TextInputFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.exec.FooterBuffer;
import org.apache.hadoop.hive.ql.io.IOContext.Comparison;
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
import org.apache.hadoop.hive.ql.plan.TableDesc;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
/** This class prepares an IOContext, and provides the ability to perform a binary search on the
* data. The binary search can be used by setting the value of inputFormatSorted in the
* MapreduceWork to true, but it should only be used if the data is going to a FilterOperator,
* which filters by comparing a value in the data with a constant, using one of the comparisons
* =, <, >, <=, >=. If the RecordReader's underlying format is an RCFile, this object can perform
* a binary search to find the block to begin reading from, and stop reading once it can be
* determined no other entries will match the filter.
*/
public abstract class HiveContextAwareRecordReader<K extends WritableComparable, V extends Writable>
implements RecordReader<K, V> {
private static final Logger LOG = LoggerFactory.getLogger(HiveContextAwareRecordReader.class.getName());
private boolean initDone = false;
private long rangeStart;
private long rangeEnd;
private long splitEnd;
private long previousPosition = -1;
private boolean wasUsingSortedSearch = false;
private String genericUDFClassName = null;
private final List<Comparison> stopComparisons = new ArrayList<Comparison>();
private Map<Path, PartitionDesc> pathToPartitionInfo;
protected RecordReader<K, V> recordReader;
protected JobConf jobConf;
protected boolean isSorted = false;
public HiveContextAwareRecordReader(JobConf conf) throws IOException {
this(null, conf);
}
public HiveContextAwareRecordReader(RecordReader<K, V> recordReader) {
this.recordReader = recordReader;
}
public HiveContextAwareRecordReader(RecordReader<K, V> recordReader, JobConf conf)
throws IOException {
this.recordReader = recordReader;
this.jobConf = conf;
}
public void setRecordReader(RecordReader<K, V> recordReader) {
this.recordReader = recordReader;
}
/**
* Close this {@link InputSplit} to future operations.
*
* @throws IOException
*/
public abstract void doClose() throws IOException;
private IOContext ioCxtRef = null;
@Override
public void close() throws IOException {
doClose();
initDone = false;
ioCxtRef = null;
}
@Override
public boolean next(K key, V value) throws IOException {
if(!initDone) {
throw new IOException("Hive IOContext is not inited.");
}
updateIOContext();
try {
boolean retVal = doNext(key, value);
if(retVal) {
if(key instanceof RecordIdentifier) {
//supports AcidInputFormat which uses the KEY pass ROW__ID info
ioCxtRef.setRecordIdentifier((RecordIdentifier)key);
}
else if(recordReader instanceof AcidInputFormat.AcidRecordReader) {
//supports AcidInputFormat which do not use the KEY pass ROW__ID info
AcidInputFormat.AcidRecordReader acidRecordReader = (AcidInputFormat.AcidRecordReader) this.recordReader;
OrcRawRecordMerger.ReaderKey recordIdentifier = acidRecordReader.getRecordIdentifier();
if (recordIdentifier == null) {
ioCxtRef.parseRecordIdentifier(jobConf);
} else {
ioCxtRef.setRecordIdentifier(recordIdentifier);
}
ioCxtRef.setDeletedRecord(recordIdentifier != null && recordIdentifier.isDeleteEvent());
} else {
ioCxtRef.parseRecordIdentifier(jobConf);
ioCxtRef.parsePositionDeleteInfo(jobConf);
}
}
return retVal;
} catch (IOException e) {
ioCxtRef.setIOExceptions(true);
throw e;
}
}
protected void updateIOContext()
throws IOException {
long pointerPos = this.getPos();
if (!ioCxtRef.isBlockPointer()) {
ioCxtRef.setCurrentBlockStart(pointerPos);
ioCxtRef.setCurrentRow(0);
return;
}
ioCxtRef.setCurrentRow(ioCxtRef.getCurrentRow() + 1);
if (ioCxtRef.getNextBlockStart() == -1) {
ioCxtRef.setNextBlockStart(pointerPos);
ioCxtRef.setCurrentRow(0);
}
if (pointerPos != ioCxtRef.getNextBlockStart()) {
// the reader pointer has moved to the end of next block, or the end of
// current record.
ioCxtRef.setCurrentRow(0);
if (ioCxtRef.getCurrentBlockStart() == ioCxtRef.getNextBlockStart()) {
ioCxtRef.setCurrentRow(1);
}
ioCxtRef.setCurrentBlockStart(ioCxtRef.getNextBlockStart());
ioCxtRef.setNextBlockStart(pointerPos);
}
}
public IOContext getIOContext() {
return IOContextMap.get(jobConf);
}
private void initIOContext(long startPos, boolean isBlockPointer,
Path inputPath) {
ioCxtRef = this.getIOContext();
ioCxtRef.setCurrentBlockStart(startPos);
ioCxtRef.setBlockPointer(isBlockPointer);
ioCxtRef.setInputPath(inputPath);
LOG.debug("Processing file " + inputPath); // Logged at INFO in multiple other places.
initDone = true;
}
public void initIOContext(FileSplit split, JobConf job,
Class inputFormatClass) throws IOException {
this.initIOContext(split, job, inputFormatClass, null);
}
public void initIOContext(FileSplit split, JobConf job,
Class inputFormatClass, RecordReader recordReader) throws IOException {
boolean blockPointer = false;
long blockStart = -1;
FileSplit fileSplit = split;
Path path = fileSplit.getPath();
if (inputFormatClass.getName().contains("SequenceFile")) {
FileSystem fs = path.getFileSystem(job);
SequenceFile.Reader in = new SequenceFile.Reader(fs, path, job);
blockPointer = in.isBlockCompressed();
in.sync(fileSplit.getStart());
blockStart = in.getPosition();
in.close();
} else if (recordReader instanceof RCFileRecordReader) {
blockPointer = true;
blockStart = ((RCFileRecordReader) recordReader).getStart();
} else if (inputFormatClass.getName().contains("RCFile")) {
FileSystem fs = path.getFileSystem(job);
blockPointer = true;
RCFile.Reader in = new RCFile.Reader(fs, path, job);
in.sync(fileSplit.getStart());
blockStart = in.getPosition();
in.close();
}
this.jobConf = job;
this.initIOContext(blockStart, blockPointer, path);
this.initIOContextSortedProps(split, recordReader, job);
}
public void initIOContextSortedProps(FileSplit split, RecordReader recordReader, JobConf job) {
this.jobConf = job;
this.getIOContext().resetSortingValues();
this.isSorted = jobConf.getBoolean("hive.input.format.sorted", false);
this.rangeStart = split.getStart();
this.rangeEnd = split.getStart() + split.getLength();
this.splitEnd = rangeEnd;
if (recordReader instanceof RCFileRecordReader && rangeEnd != 0 && this.isSorted) {
// Binary search only works if we know the size of the split, and the recordReader is an
// RCFileRecordReader
this.getIOContext().setUseSorted(true);
this.getIOContext().setBinarySearching(true);
this.wasUsingSortedSearch = true;
} else {
// Use the defalut methods for next in the child class
this.isSorted = false;
}
}
@Override
public float getProgress() throws IOException {
if (this.getIOContext().isBinarySearching()) {
return 0;
} else {
return recordReader.getProgress();
}
}
private FooterBuffer footerBuffer = null;
private int headerCount = 0;
private int footerCount = 0;
protected FooterBuffer getFooterBuffer() {
return footerBuffer;
}
protected void setFooterBuffer( FooterBuffer buf) {
footerBuffer = buf;
}
public boolean doNext(K key, V value) throws IOException {
if (this.isSorted) {
if (this.getIOContext().shouldEndBinarySearch() ||
(!this.getIOContext().useSorted() && this.wasUsingSortedSearch)) {
beginLinearSearch();
this.wasUsingSortedSearch = false;
this.getIOContext().setEndBinarySearch(false);
}
if (this.getIOContext().useSorted()) {
if (this.genericUDFClassName == null &&
this.getIOContext().getGenericUDFClassName() != null) {
setGenericUDFClassName(this.getIOContext().getGenericUDFClassName());
}
if (this.getIOContext().isBinarySearching()) {
// Proceed with a binary search
if (this.getIOContext().getComparison() != null) {
switch (this.getIOContext().getComparison()) {
case GREATER:
case EQUAL:
// Indexes have only one entry per value, could go linear from here, if we want to
// use this for any sorted table, we'll need to continue the search
rangeEnd = previousPosition;
break;
case LESS:
rangeStart = previousPosition;
break;
default:
break;
}
}
long position = (rangeStart + rangeEnd) / 2;
sync(position);
long newPosition = getSyncedPosition();
// If the newPosition is the same as the previousPosition, we've reached the end of the
// binary search, if the new position at least as big as the size of the split, any
// matching rows must be in the final block, so we can end the binary search.
if (newPosition == previousPosition || newPosition >= splitEnd) {
this.getIOContext().setBinarySearching(false);
sync(rangeStart);
}
previousPosition = newPosition;
} else if (foundAllTargets()) {
// Found all possible rows which will not be filtered
return false;
}
}
}
try {
/**
* When start reading new file, check header, footer rows.
* If file contains header, skip header lines before reading the records.
* If file contains footer, used a FooterBuffer to remove footer lines
* at the end of the table file.
**/
if (this.ioCxtRef.getCurrentBlockStart() == 0) {
// Check if the table file has header to skip.
footerBuffer = null;
Path filePath = this.ioCxtRef.getInputPath();
PartitionDesc part = null;
try {
if (pathToPartitionInfo == null) {
pathToPartitionInfo = Utilities
.getMapWork(jobConf).getPathToPartitionInfo();
}
part = HiveFileFormatUtils
.getFromPathRecursively(pathToPartitionInfo,
filePath, IOPrepareCache.get().getPartitionDescMap());
} catch (AssertionError ae) {
LOG.info("Cannot get partition description from " + this.ioCxtRef.getInputPath()
+ "because " + ae.getMessage());
part = null;
} catch (Exception e) {
LOG.info("Cannot get partition description from " + this.ioCxtRef.getInputPath()
+ "because " + e.getMessage());
part = null;
}
TableDesc table = (part == null) ? null : part.getTableDesc();
// In TextFormat, skipping is already taken care of as part of SkippingTextInputFormat.
// This code will be also called from LLAP when pipeline is non-vectorized and cannot create wrapper.
if (table != null && !TextInputFormat.class.isAssignableFrom(part.getInputFileFormatClass())) {
headerCount = Utilities.getHeaderCount(table);
footerCount = Utilities.getFooterCount(table, jobConf);
}
// If input contains header, skip header.
if (!Utilities.skipHeader(recordReader, headerCount, key, value)) {
return false;
}
if (footerCount > 0) {
footerBuffer = new FooterBuffer();
if (!footerBuffer.initializeBuffer(jobConf, recordReader, footerCount, key, value)) {
return false;
}
}
}
if (footerBuffer == null) {
// Table files don't have footer rows.
return recordReader.next(key, value);
} else {
return footerBuffer.updateBuffer(jobConf, recordReader, key, value);
}
} catch (Exception e) {
return HiveIOExceptionHandlerUtil.handleRecordReaderNextException(e, jobConf);
}
}
private void sync(long position) throws IOException {
((RCFileRecordReader)recordReader).sync(position);
((RCFileRecordReader)recordReader).resetBuffer();
}
private long getSyncedPosition() throws IOException {
return recordReader.getPos();
}
/**
* This uses the name of the generic UDF being used by the filter to determine whether we should
* perform a binary search, and what the comparisons we should use to signal the end of the
* linear scan are.
* @param genericUDFClassName
* @throws IOException
*/
private void setGenericUDFClassName(String genericUDFClassName) throws IOException {
this.genericUDFClassName = genericUDFClassName;
if (genericUDFClassName.equals(GenericUDFOPEqual.class.getName())) {
stopComparisons.add(Comparison.GREATER);
} else if (genericUDFClassName.equals(GenericUDFOPLessThan.class.getName())) {
stopComparisons.add(Comparison.EQUAL);
stopComparisons.add(Comparison.GREATER);
if (this.getIOContext().isBinarySearching()) {
beginLinearSearch();
}
} else if (genericUDFClassName.equals(GenericUDFOPEqualOrLessThan.class.getName())) {
stopComparisons.add(Comparison.GREATER);
if (this.getIOContext().isBinarySearching()) {
beginLinearSearch();
}
} else if (genericUDFClassName.equals(GenericUDFOPGreaterThan.class.getName()) ||
genericUDFClassName.equals(GenericUDFOPEqualOrGreaterThan.class.getName())) {
// Do nothing
} else {
// This is an unsupported operator
LOG.debug(genericUDFClassName + " is not the name of a supported class. " +
"Continuing linearly.");
if (this.getIOContext().isBinarySearching()) {
beginLinearSearch();
}
}
}
/**
* This should be called after the binary search is finished and before the linear scan begins
* @throws IOException
*/
private void beginLinearSearch() throws IOException {
sync(rangeStart);
this.getIOContext().setBinarySearching(false);
this.wasUsingSortedSearch = false;
}
/**
* Returns true if the current comparison is in the list of stop comparisons, i.e. we've found
* all records which won't be filtered
* @return true if the current comparison is found
*/
public boolean foundAllTargets() {
if (this.getIOContext().getComparison() == null ||
!stopComparisons.contains(this.getIOContext().getComparison())) {
return false;
}
return true;
}
}