|
@@ -35,10 +35,12 @@ import org.apache.hadoop.hbase.CellUtil;
|
|
|
import org.apache.hadoop.hbase.HRegionInfo;
|
|
|
import org.apache.hadoop.hbase.KeyValue;
|
|
|
import org.apache.hadoop.hbase.Tag;
|
|
|
+import org.apache.hadoop.hbase.client.Scan;
|
|
|
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
|
|
-import org.apache.hadoop.hbase.regionserver.HRegion;
|
|
|
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
|
|
+import org.apache.hadoop.hbase.regionserver.Region;
|
|
|
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
|
|
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
|
|
import org.apache.hadoop.hbase.util.Bytes;
|
|
|
import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -68,9 +70,9 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
*/
|
|
|
private static final String FLOW_APP_ID = "application_00000000000_0000";
|
|
|
|
|
|
- private final HRegion region;
|
|
|
+ private final Region region;
|
|
|
private final InternalScanner flowRunScanner;
|
|
|
- private final int limit;
|
|
|
+ private final int batchSize;
|
|
|
private final long appFinalValueRetentionThreshold;
|
|
|
private RegionScanner regionScanner;
|
|
|
private boolean hasMore;
|
|
@@ -79,9 +81,15 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
private int currentIndex;
|
|
|
private FlowScannerOperation action = FlowScannerOperation.READ;
|
|
|
|
|
|
- FlowScanner(RegionCoprocessorEnvironment env, int limit,
|
|
|
+ FlowScanner(RegionCoprocessorEnvironment env, InternalScanner internalScanner,
|
|
|
+ FlowScannerOperation action) {
|
|
|
+ this(env, null, internalScanner, action);
|
|
|
+ }
|
|
|
+
|
|
|
+ FlowScanner(RegionCoprocessorEnvironment env, Scan incomingScan,
|
|
|
InternalScanner internalScanner, FlowScannerOperation action) {
|
|
|
- this.limit = limit;
|
|
|
+ this.batchSize = incomingScan == null ? -1 : incomingScan.getBatch();
|
|
|
+ // TODO initialize other scan attributes like Scan#maxResultSize
|
|
|
this.flowRunScanner = internalScanner;
|
|
|
if (internalScanner instanceof RegionScanner) {
|
|
|
this.regionScanner = (RegionScanner) internalScanner;
|
|
@@ -98,8 +106,12 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
YarnConfiguration.APP_FINAL_VALUE_RETENTION_THRESHOLD,
|
|
|
YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD);
|
|
|
}
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug(" batch size=" + batchSize);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
+
|
|
|
/*
|
|
|
* (non-Javadoc)
|
|
|
*
|
|
@@ -112,22 +124,24 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
|
|
|
@Override
|
|
|
public boolean nextRaw(List<Cell> cells) throws IOException {
|
|
|
- return nextRaw(cells, limit);
|
|
|
+ return nextRaw(cells, ScannerContext.newBuilder().build());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public boolean nextRaw(List<Cell> cells, int cellLimit) throws IOException {
|
|
|
- return nextInternal(cells, cellLimit);
|
|
|
+ public boolean nextRaw(List<Cell> cells, ScannerContext scannerContext)
|
|
|
+ throws IOException {
|
|
|
+ return nextInternal(cells, scannerContext);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public boolean next(List<Cell> cells) throws IOException {
|
|
|
- return next(cells, limit);
|
|
|
+ return next(cells, ScannerContext.newBuilder().build());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public boolean next(List<Cell> cells, int cellLimit) throws IOException {
|
|
|
- return nextInternal(cells, cellLimit);
|
|
|
+ public boolean next(List<Cell> cells, ScannerContext scannerContext)
|
|
|
+ throws IOException {
|
|
|
+ return nextInternal(cells, scannerContext);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -158,17 +172,6 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
return GenericConverter.getInstance();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Checks if the converter is a numeric converter or not. For a converter to
|
|
|
- * be numeric, it must implement {@link NumericValueConverter} interface.
|
|
|
- * @param converter
|
|
|
- * @return true, if converter is of type NumericValueConverter, false
|
|
|
- * otherwise.
|
|
|
- */
|
|
|
- private static boolean isNumericConverter(ValueConverter converter) {
|
|
|
- return (converter instanceof NumericValueConverter);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* This method loops through the cells in a given row of the
|
|
|
* {@link FlowRunTable}. It looks at the tags of each cell to figure out how
|
|
@@ -176,12 +179,11 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
* column or returns the cell as is.
|
|
|
*
|
|
|
* @param cells
|
|
|
- * @param cellLimit
|
|
|
+ * @param scannerContext
|
|
|
* @return true if next row is available for the scanner, false otherwise
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- private boolean nextInternal(List<Cell> cells, int cellLimit)
|
|
|
+ private boolean nextInternal(List<Cell> cells, ScannerContext scannerContext)
|
|
|
throws IOException {
|
|
|
Cell cell = null;
|
|
|
startNext();
|
|
@@ -194,48 +196,47 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
// So all cells in one qualifier come one after the other before we see the
|
|
|
// next column qualifier
|
|
|
ByteArrayComparator comp = new ByteArrayComparator();
|
|
|
- byte[] currentColumnQualifier = Separator.EMPTY_BYTES;
|
|
|
+ byte[] previousColumnQualifier = Separator.EMPTY_BYTES;
|
|
|
AggregationOperation currentAggOp = null;
|
|
|
SortedSet<Cell> currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR);
|
|
|
Set<String> alreadySeenAggDim = new HashSet<>();
|
|
|
int addedCnt = 0;
|
|
|
long currentTimestamp = System.currentTimeMillis();
|
|
|
ValueConverter converter = null;
|
|
|
+ int limit = batchSize;
|
|
|
|
|
|
- while (cellLimit <= 0 || addedCnt < cellLimit) {
|
|
|
- cell = peekAtNextCell(cellLimit);
|
|
|
+ while (limit <= 0 || addedCnt < limit) {
|
|
|
+ cell = peekAtNextCell(scannerContext);
|
|
|
if (cell == null) {
|
|
|
break;
|
|
|
}
|
|
|
- byte[] newColumnQualifier = CellUtil.cloneQualifier(cell);
|
|
|
- if (comp.compare(currentColumnQualifier, newColumnQualifier) != 0) {
|
|
|
- if (converter != null && isNumericConverter(converter)) {
|
|
|
- addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
|
|
|
- converter, currentTimestamp);
|
|
|
- }
|
|
|
+ byte[] currentColumnQualifier = CellUtil.cloneQualifier(cell);
|
|
|
+ if (previousColumnQualifier == null) {
|
|
|
+ // first time in loop
|
|
|
+ previousColumnQualifier = currentColumnQualifier;
|
|
|
+ }
|
|
|
+
|
|
|
+ converter = getValueConverter(currentColumnQualifier);
|
|
|
+ if (comp.compare(previousColumnQualifier, currentColumnQualifier) != 0) {
|
|
|
+ addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
|
|
|
+ converter, currentTimestamp);
|
|
|
resetState(currentColumnCells, alreadySeenAggDim);
|
|
|
- currentColumnQualifier = newColumnQualifier;
|
|
|
+ previousColumnQualifier = currentColumnQualifier;
|
|
|
currentAggOp = getCurrentAggOp(cell);
|
|
|
- converter = getValueConverter(newColumnQualifier);
|
|
|
- }
|
|
|
- // No operation needs to be performed on non numeric converters.
|
|
|
- if (!isNumericConverter(converter)) {
|
|
|
- currentColumnCells.add(cell);
|
|
|
- nextCell(cellLimit);
|
|
|
- continue;
|
|
|
+ converter = getValueConverter(currentColumnQualifier);
|
|
|
}
|
|
|
collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim,
|
|
|
- (NumericValueConverter)converter);
|
|
|
- nextCell(cellLimit);
|
|
|
+ converter, scannerContext);
|
|
|
+ nextCell(scannerContext);
|
|
|
}
|
|
|
- if (!currentColumnCells.isEmpty()) {
|
|
|
- addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
|
|
|
- converter, currentTimestamp);
|
|
|
+ if ((!currentColumnCells.isEmpty()) && ((limit <= 0 || addedCnt < limit))) {
|
|
|
+ addedCnt += emitCells(cells, currentColumnCells, currentAggOp, converter,
|
|
|
+ currentTimestamp);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
if (addedCnt > 0) {
|
|
|
LOG.debug("emitted cells. " + addedCnt + " for " + this.action
|
|
|
+ " rowKey="
|
|
|
- + FlowRunRowKey.parseRowKey(cells.get(0).getRow()).toString());
|
|
|
+ + FlowRunRowKey.parseRowKey(CellUtil.cloneRow(cells.get(0))));
|
|
|
} else {
|
|
|
LOG.debug("emitted no cells for " + this.action);
|
|
|
}
|
|
@@ -252,7 +253,7 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * resets the parameters to an intialized state for next loop iteration.
|
|
|
+ * resets the parameters to an initialized state for next loop iteration.
|
|
|
*
|
|
|
* @param cell
|
|
|
* @param currentAggOp
|
|
@@ -268,12 +269,12 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
|
|
|
private void collectCells(SortedSet<Cell> currentColumnCells,
|
|
|
AggregationOperation currentAggOp, Cell cell,
|
|
|
- Set<String> alreadySeenAggDim, NumericValueConverter converter)
|
|
|
- throws IOException {
|
|
|
+ Set<String> alreadySeenAggDim, ValueConverter converter,
|
|
|
+ ScannerContext scannerContext) throws IOException {
|
|
|
+
|
|
|
if (currentAggOp == null) {
|
|
|
// not a min/max/metric cell, so just return it as is
|
|
|
currentColumnCells.add(cell);
|
|
|
- nextCell(limit);
|
|
|
return;
|
|
|
}
|
|
|
|
|
@@ -284,7 +285,7 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
} else {
|
|
|
Cell currentMinCell = currentColumnCells.first();
|
|
|
Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp,
|
|
|
- converter);
|
|
|
+ (NumericValueConverter) converter);
|
|
|
if (!currentMinCell.equals(newMinCell)) {
|
|
|
currentColumnCells.remove(currentMinCell);
|
|
|
currentColumnCells.add(newMinCell);
|
|
@@ -297,7 +298,7 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
} else {
|
|
|
Cell currentMaxCell = currentColumnCells.first();
|
|
|
Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp,
|
|
|
- converter);
|
|
|
+ (NumericValueConverter) converter);
|
|
|
if (!currentMaxCell.equals(newMaxCell)) {
|
|
|
currentColumnCells.remove(currentMaxCell);
|
|
|
currentColumnCells.add(newMaxCell);
|
|
@@ -610,15 +611,14 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
* pointer to the next cell. This method can be called multiple times in a row
|
|
|
* to advance through all the available cells.
|
|
|
*
|
|
|
- * @param cellLimit
|
|
|
- * the limit of number of cells to return if the next batch must be
|
|
|
- * fetched by the wrapped scanner
|
|
|
+ * @param scannerContext
|
|
|
+ * context information for the batch of cells under consideration
|
|
|
* @return the next available cell or null if no more cells are available for
|
|
|
* the current row
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public Cell nextCell(int cellLimit) throws IOException {
|
|
|
- Cell cell = peekAtNextCell(cellLimit);
|
|
|
+ public Cell nextCell(ScannerContext scannerContext) throws IOException {
|
|
|
+ Cell cell = peekAtNextCell(scannerContext);
|
|
|
if (cell != null) {
|
|
|
currentIndex++;
|
|
|
}
|
|
@@ -630,20 +630,19 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
* pointer. Calling this method multiple times in a row will continue to
|
|
|
* return the same cell.
|
|
|
*
|
|
|
- * @param cellLimit
|
|
|
- * the limit of number of cells to return if the next batch must be
|
|
|
- * fetched by the wrapped scanner
|
|
|
+ * @param scannerContext
|
|
|
+ * context information for the batch of cells under consideration
|
|
|
* @return the next available cell or null if no more cells are available for
|
|
|
* the current row
|
|
|
* @throws IOException if any problem is encountered while grabbing the next
|
|
|
* cell.
|
|
|
*/
|
|
|
- public Cell peekAtNextCell(int cellLimit) throws IOException {
|
|
|
+ public Cell peekAtNextCell(ScannerContext scannerContext) throws IOException {
|
|
|
if (currentIndex >= availableCells.size()) {
|
|
|
// done with current batch
|
|
|
availableCells.clear();
|
|
|
currentIndex = 0;
|
|
|
- hasMore = flowRunScanner.next(availableCells, cellLimit);
|
|
|
+ hasMore = flowRunScanner.next(availableCells, scannerContext);
|
|
|
}
|
|
|
Cell cell = null;
|
|
|
if (currentIndex < availableCells.size()) {
|
|
@@ -720,4 +719,9 @@ class FlowScanner implements RegionScanner, Closeable {
|
|
|
}
|
|
|
return regionScanner.reseek(bytes);
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int getBatch() {
|
|
|
+ return batchSize;
|
|
|
+ }
|
|
|
}
|