in phoenix5-hive4/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordReader.java [85:145]
public void initialize(InputSplit split) throws IOException {
final PhoenixInputSplit pSplit = (PhoenixInputSplit) split;
final List<Scan> scans = pSplit.getScans();
if (LOG.isInfoEnabled()) {
LOG.info("Target table : " + queryPlan.getTableRef().getTable().getPhysicalName());
}
if (LOG.isDebugEnabled()) {
LOG.debug("Scan count[" + scans.size() + "] : " + Bytes.toStringBinary(scans.get(0)
.getStartRow()) + " ~ " + Bytes.toStringBinary(scans.get(scans.size() - 1)
.getStopRow()));
LOG.debug("First scan : " + scans.get(0) + " scanAttribute : " + scans.get(0)
.getAttributesMap());
for (int i = 0, limit = scans.size(); i < limit; i++) {
LOG.debug("EXPECTED_UPPER_REGION_KEY[" + i + "] : " +
Bytes.toStringBinary(scans.get(i).getAttribute(
BaseScannerRegionObserverConstants.EXPECTED_UPPER_REGION_KEY)));
}
}
try {
List<PeekingResultIterator> iterators = new ArrayList<>(scans.size
());
StatementContext ctx = queryPlan.getContext();
ReadMetricQueue readMetrics = ctx.getReadMetricsQueue();
String tableName = queryPlan.getTableRef().getTable().getPhysicalName().getString();
long renewScannerLeaseThreshold = queryPlan.getContext().getConnection()
.getQueryServices().getRenewLeaseThresholdMilliSeconds();
for (Scan scan : scans) {
scan.setAttribute(BaseScannerRegionObserverConstants.SKIP_REGION_BOUNDARY_CHECK,
Bytes.toBytes(true));
ScanMetricsHolder scanMetricsHolder = ScanMetricsHolder.getInstance(readMetrics, tableName, scan, ctx.getConnection().getLogLevel());
final TableResultIterator tableResultIterator = new TableResultIterator(
queryPlan.getContext().getConnection().getMutationState(), scan, scanMetricsHolder,
renewScannerLeaseThreshold, queryPlan, MapReduceParallelScanGrouper.getInstance());
PeekingResultIterator peekingResultIterator = LookAheadResultIterator.wrap
(tableResultIterator);
iterators.add(peekingResultIterator);
}
ResultIterator iterator = queryPlan.useRoundRobinIterator()
? RoundRobinResultIterator.newIterator(iterators, queryPlan)
: ConcatResultIterator.newIterator(iterators);
if (queryPlan.getContext().getSequenceManager().getSequenceCount() > 0) {
iterator = new SequenceResultIterator(iterator, queryPlan.getContext()
.getSequenceManager());
}
this.resultIterator = iterator;
// Clone the row projector as it's not thread safe and would be used
// simultaneously by multiple threads otherwise.
this.resultSet = new PhoenixResultSet(this.resultIterator, queryPlan.getProjector()
.cloneIfNecessary(),
queryPlan.getContext());
} catch (SQLException e) {
LOG.error(String.format(" Error [%s] initializing PhoenixRecordReader. ", e
.getMessage()));
Throwables.propagate(e);
}
}