http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java deleted file mode 100644 index 7d6beab..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java +++ /dev/null @@ -1,528 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.PrivateCellUtil; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.codec.prefixtree.decode.column.ColumnReader; -import org.apache.hadoop.hbase.codec.prefixtree.decode.row.RowNodeReader; -import org.apache.hadoop.hbase.codec.prefixtree.decode.timestamp.MvccVersionDecoder; -import org.apache.hadoop.hbase.codec.prefixtree.decode.timestamp.TimestampDecoder; -import org.apache.hadoop.hbase.codec.prefixtree.encode.other.ColumnNodeType; -import org.apache.hadoop.hbase.nio.ByteBuff; - -/** - * Extends PtCell and manipulates its protected fields. Could alternatively contain a PtCell and - * call get/set methods. - * - * This is an "Array" scanner to distinguish from a future "ByteBuffer" scanner. This - * implementation requires that the bytes be in a normal java byte[] for performance. The - * alternative ByteBuffer implementation would allow for accessing data in an off-heap ByteBuffer - * without copying the whole buffer on-heap. - */ -@InterfaceAudience.Private -public class PrefixTreeArrayScanner extends PrefixTreeCell implements CellScanner { - - /***************** fields ********************************/ - - protected PrefixTreeBlockMeta blockMeta; - - protected boolean beforeFirst; - protected boolean afterLast; - - protected RowNodeReader[] rowNodes; - protected int rowNodeStackIndex; - - protected RowNodeReader currentRowNode; - protected ColumnReader familyReader; - protected ColumnReader qualifierReader; - protected ColumnReader tagsReader; - protected TimestampDecoder timestampDecoder; - protected MvccVersionDecoder mvccVersionDecoder; - - protected boolean nubCellsRemain; - protected int currentCellIndex; - - /*********************** construct ******************************/ - - // pass in blockMeta so we can initialize buffers big enough for all cells in the block - public PrefixTreeArrayScanner(PrefixTreeBlockMeta blockMeta, int rowTreeDepth, - int rowBufferLength, int qualifierBufferLength, int tagsBufferLength) { - this.rowNodes = new RowNodeReader[rowTreeDepth]; - for (int i = 0; i < rowNodes.length; ++i) { - rowNodes[i] = new RowNodeReader(); - } - this.rowBuffer = new byte[rowBufferLength]; - this.familyBuffer = new byte[PrefixTreeBlockMeta.MAX_FAMILY_LENGTH]; - this.familyReader = new ColumnReader(familyBuffer, ColumnNodeType.FAMILY); - this.qualifierBuffer = new byte[qualifierBufferLength]; - this.tagsBuffer = new byte[tagsBufferLength]; - this.qualifierReader = new ColumnReader(qualifierBuffer, ColumnNodeType.QUALIFIER); - this.tagsReader = new ColumnReader(tagsBuffer, ColumnNodeType.TAGS); - this.timestampDecoder = new TimestampDecoder(); - this.mvccVersionDecoder = new MvccVersionDecoder(); - } - - - /**************** init helpers ***************************************/ - - /** - * Call when first accessing a block. - * @return entirely new scanner if false - */ - public boolean areBuffersBigEnough() { - if (rowNodes.length < blockMeta.getRowTreeDepth()) { - return false; - } - if (rowBuffer.length < blockMeta.getMaxRowLength()) { - return false; - } - if (qualifierBuffer.length < blockMeta.getMaxQualifierLength()) { - return false; - } - if(tagsBuffer.length < blockMeta.getMaxTagsLength()) { - return false; - } - return true; - } - - public void initOnBlock(PrefixTreeBlockMeta blockMeta, ByteBuff block, - boolean includeMvccVersion) { - this.block = block; - this.blockMeta = blockMeta; - this.familyOffset = familyBuffer.length; - this.familyReader.initOnBlock(blockMeta, block); - this.qualifierOffset = qualifierBuffer.length; - this.qualifierReader.initOnBlock(blockMeta, block); - this.tagsOffset = tagsBuffer.length; - this.tagsReader.initOnBlock(blockMeta, block); - this.timestampDecoder.initOnBlock(blockMeta, block); - this.mvccVersionDecoder.initOnBlock(blockMeta, block); - this.includeMvccVersion = includeMvccVersion; - resetToBeforeFirstEntry(); - } - - // Does this have to be in the CellScanner Interface? TODO - public void resetToBeforeFirstEntry() { - beforeFirst = true; - afterLast = false; - rowNodeStackIndex = -1; - currentRowNode = null; - rowLength = 0; - familyOffset = familyBuffer.length; - familyLength = 0; - qualifierOffset = blockMeta.getMaxQualifierLength(); - qualifierLength = 0; - nubCellsRemain = false; - currentCellIndex = -1; - timestamp = -1L; - type = DEFAULT_TYPE; - absoluteValueOffset = 0;//use 0 vs -1 so the cell is valid when value hasn't been initialized - valueLength = 0;// had it at -1, but that causes null Cell to add up to the wrong length - tagsOffset = blockMeta.getMaxTagsLength(); - tagsLength = 0; - } - - /** - * Call this before putting the scanner back into a pool so it doesn't hold the last used block - * in memory. - */ - public void releaseBlockReference(){ - block = null; - } - - - /********************** CellScanner **********************/ - - @Override - public Cell current() { - if(isOutOfBounds()){ - return null; - } - return (Cell)this; - } - - /******************* Object methods ************************/ - - @Override - public boolean equals(Object obj) { - //trivial override to confirm intent (findbugs) - return super.equals(obj); - } - - @Override - public int hashCode() { - return super.hashCode(); - } - - /** - * Override PrefixTreeCell.toString() with a check to see if the current cell is valid. - */ - @Override - public String toString() { - Cell currentCell = current(); - if(currentCell==null){ - return "null"; - } - return ((PrefixTreeCell)currentCell).getKeyValueString(); - } - - - /******************* advance ***************************/ - - public boolean positionAtFirstCell() { - reInitFirstNode(); - return advance(); - } - - @Override - public boolean advance() { - if (afterLast) { - return false; - } - if (!hasOccurrences()) { - resetToBeforeFirstEntry(); - } - if (beforeFirst || isLastCellInRow()) { - nextRow(); - if (afterLast) { - return false; - } - } else { - ++currentCellIndex; - } - - populateNonRowFields(currentCellIndex); - return true; - } - - - public boolean nextRow() { - nextRowInternal(); - if (afterLast) { - return false; - } - populateNonRowFields(currentCellIndex); - return true; - } - - - /** - * This method is safe to call when the scanner is not on a fully valid row node, as in the case - * of a row token miss in the Searcher - * @return true if we are positioned on a valid row, false if past end of block - */ - protected boolean nextRowInternal() { - if (afterLast) { - return false; - } - if (beforeFirst) { - initFirstNode(); - if (currentRowNode.hasOccurrences()) { - if (currentRowNode.isNub()) { - nubCellsRemain = true; - } - currentCellIndex = 0; - return true; - } - } - if (currentRowNode.isLeaf()) { - discardCurrentRowNode(true); - } - while (!afterLast) { - if (nubCellsRemain) { - nubCellsRemain = false; - } - if (currentRowNode.hasMoreFanNodes()) { - followNextFan(); - if (currentRowNode.hasOccurrences()) { - // found some values - currentCellIndex = 0; - return true; - } - } else { - discardCurrentRowNode(true); - } - } - return false;// went past the end - } - - - /**************** secondary traversal methods ******************************/ - - protected void reInitFirstNode() { - resetToBeforeFirstEntry(); - initFirstNode(); - } - - protected void initFirstNode() { - int offsetIntoUnderlyingStructure = blockMeta.getAbsoluteRowOffset(); - rowNodeStackIndex = 0; - currentRowNode = rowNodes[0]; - currentRowNode.initOnBlock(blockMeta, block, offsetIntoUnderlyingStructure); - appendCurrentTokenToRowBuffer(); - beforeFirst = false; - } - - protected void followFirstFan() { - followFan(0); - } - - protected void followPreviousFan() { - int nextFanPosition = currentRowNode.getFanIndex() - 1; - followFan(nextFanPosition); - } - - protected void followCurrentFan() { - int currentFanPosition = currentRowNode.getFanIndex(); - followFan(currentFanPosition); - } - - protected void followNextFan() { - int nextFanPosition = currentRowNode.getFanIndex() + 1; - followFan(nextFanPosition); - } - - protected void followLastFan() { - followFan(currentRowNode.getLastFanIndex()); - } - - protected void followFan(int fanIndex) { - currentRowNode.setFanIndex(fanIndex); - appendToRowBuffer(currentRowNode.getFanByte(fanIndex)); - - int nextOffsetIntoUnderlyingStructure = currentRowNode.getOffset() - + currentRowNode.getNextNodeOffset(fanIndex, blockMeta); - ++rowNodeStackIndex; - - currentRowNode = rowNodes[rowNodeStackIndex]; - currentRowNode.initOnBlock(blockMeta, block, nextOffsetIntoUnderlyingStructure); - - //TODO getToken is spewing garbage - appendCurrentTokenToRowBuffer(); - if (currentRowNode.isNub()) { - nubCellsRemain = true; - } - currentCellIndex = 0; - } - - /** - * @param forwards which marker to set if we overflow - */ - protected void discardCurrentRowNode(boolean forwards) { - RowNodeReader rowNodeBeingPopped = currentRowNode; - --rowNodeStackIndex;// pop it off the stack - if (rowNodeStackIndex < 0) { - currentRowNode = null; - if (forwards) { - markAfterLast(); - } else { - markBeforeFirst(); - } - return; - } - popFromRowBuffer(rowNodeBeingPopped); - currentRowNode = rowNodes[rowNodeStackIndex]; - } - - protected void markBeforeFirst() { - beforeFirst = true; - afterLast = false; - currentRowNode = null; - } - - protected void markAfterLast() { - beforeFirst = false; - afterLast = true; - currentRowNode = null; - } - - - /***************** helper methods **************************/ - - protected void appendCurrentTokenToRowBuffer() { - block.get(currentRowNode.getTokenArrayOffset(), rowBuffer, rowLength, - currentRowNode.getTokenLength()); - rowLength += currentRowNode.getTokenLength(); - } - - protected void appendToRowBuffer(byte b) { - rowBuffer[rowLength] = b; - ++rowLength; - } - - protected void popFromRowBuffer(RowNodeReader rowNodeBeingPopped) { - rowLength -= rowNodeBeingPopped.getTokenLength(); - --rowLength; // pop the parent's fan byte - } - - protected boolean hasOccurrences() { - return currentRowNode != null && currentRowNode.hasOccurrences(); - } - - protected boolean isBranch() { - return currentRowNode != null && !currentRowNode.hasOccurrences() - && currentRowNode.hasChildren(); - } - - protected boolean isNub() { - return currentRowNode != null && currentRowNode.hasOccurrences() - && currentRowNode.hasChildren(); - } - - protected boolean isLeaf() { - return currentRowNode != null && currentRowNode.hasOccurrences() - && !currentRowNode.hasChildren(); - } - - //TODO expose this in a PrefixTreeScanner interface - public boolean isBeforeFirst(){ - return beforeFirst; - } - - public boolean isAfterLast(){ - return afterLast; - } - - protected boolean isOutOfBounds(){ - return beforeFirst || afterLast; - } - - protected boolean isFirstCellInRow() { - return currentCellIndex == 0; - } - - protected boolean isLastCellInRow() { - return currentCellIndex == currentRowNode.getLastCellIndex(); - } - - - /********************* fill in family/qualifier/ts/type/value ************/ - - protected int populateNonRowFieldsAndCompareTo(int cellNum, Cell key) { - populateNonRowFields(cellNum); - return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, this, key); - } - - protected void populateFirstNonRowFields() { - populateNonRowFields(0); - } - - protected void populatePreviousNonRowFields() { - populateNonRowFields(currentCellIndex - 1); - } - - protected void populateLastNonRowFields() { - populateNonRowFields(currentRowNode.getLastCellIndex()); - } - - protected void populateNonRowFields(int cellIndex) { - currentCellIndex = cellIndex; - populateFamily(); - populateQualifier(); - // Read tags only if there are tags in the meta - if(blockMeta.getNumTagsBytes() != 0) { - populateTag(); - } - populateTimestamp(); - populateMvccVersion(); - populateType(); - populateValueOffsets(); - } - - protected void populateFamily() { - int familyTreeIndex = currentRowNode.getFamilyOffset(currentCellIndex, blockMeta); - familyOffset = familyReader.populateBuffer(familyTreeIndex).getColumnOffset(); - familyLength = familyReader.getColumnLength(); - } - - protected void populateQualifier() { - int qualifierTreeIndex = currentRowNode.getColumnOffset(currentCellIndex, blockMeta); - qualifierOffset = qualifierReader.populateBuffer(qualifierTreeIndex).getColumnOffset(); - qualifierLength = qualifierReader.getColumnLength(); - } - - protected void populateTag() { - int tagTreeIndex = currentRowNode.getTagOffset(currentCellIndex, blockMeta); - tagsOffset = tagsReader.populateBuffer(tagTreeIndex).getColumnOffset(); - tagsLength = tagsReader.getColumnLength(); - } - - protected void populateTimestamp() { - if (blockMeta.isAllSameTimestamp()) { - timestamp = blockMeta.getMinTimestamp(); - } else { - int timestampIndex = currentRowNode.getTimestampIndex(currentCellIndex, blockMeta); - timestamp = timestampDecoder.getLong(timestampIndex); - } - } - - protected void populateMvccVersion() { - if (blockMeta.isAllSameMvccVersion()) { - mvccVersion = blockMeta.getMinMvccVersion(); - } else { - int mvccVersionIndex = currentRowNode.getMvccVersionIndex(currentCellIndex, - blockMeta); - mvccVersion = mvccVersionDecoder.getMvccVersion(mvccVersionIndex); - } - } - - protected void populateType() { - int typeInt; - if (blockMeta.isAllSameType()) { - typeInt = blockMeta.getAllTypes(); - } else { - typeInt = currentRowNode.getType(currentCellIndex, blockMeta); - } - type = PrefixTreeCell.TYPES[typeInt]; - } - - protected void populateValueOffsets() { - int offsetIntoValueSection = currentRowNode.getValueOffset(currentCellIndex, blockMeta); - absoluteValueOffset = blockMeta.getAbsoluteValueOffset() + offsetIntoValueSection; - valueLength = currentRowNode.getValueLength(currentCellIndex, blockMeta); - this.block.asSubByteBuffer(this.absoluteValueOffset, valueLength, pair); - } - - /**************** getters ***************************/ - - public PrefixTreeBlockMeta getBlockMeta() { - return blockMeta; - } - - public int getMaxRowTreeStackNodes() { - return rowNodes.length; - } - - public int getRowBufferLength() { - return rowBuffer.length; - } - - public int getQualifierBufferLength() { - return qualifierBuffer.length; - } - - public int getTagBufferLength() { - return tagsBuffer.length; - } -}
http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java deleted file mode 100644 index 7eb2517..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java +++ /dev/null @@ -1,418 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.PrivateCellUtil; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition; -import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher; - -import org.apache.hadoop.hbase.shaded.com.google.common.primitives.UnsignedBytes; - -/** - * <p> - * Searcher extends the capabilities of the Scanner + ReversibleScanner to add the ability to - * position itself on a requested Cell without scanning through cells before it. The PrefixTree is - * set up to be a Trie of rows, so finding a particular row is extremely cheap. - * </p> - * Once it finds the row, it does a binary search through the cells inside the row, which is not as - * fast as the trie search, but faster than iterating through every cell like existing block - * formats - * do. For this reason, this implementation is targeted towards schemas where rows are narrow - * enough - * to have several or many per block, and where you are generally looking for the entire row or - * the - * first cell. It will still be fast for wide rows or point queries, but could be improved upon. - */ -@InterfaceAudience.Private -public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner implements - CellSearcher { - - /*************** construct ******************************/ - - public PrefixTreeArraySearcher(PrefixTreeBlockMeta blockMeta, int rowTreeDepth, - int rowBufferLength, int qualifierBufferLength, int tagsBufferLength) { - super(blockMeta, rowTreeDepth, rowBufferLength, qualifierBufferLength, tagsBufferLength); - } - - - /********************* CellSearcher methods *******************/ - - @Override - public boolean positionAt(Cell key) { - return CellScannerPosition.AT == positionAtOrAfter(key); - } - - @Override - public CellScannerPosition positionAtOrBefore(Cell key) { - reInitFirstNode(); - int fanIndex = -1; - - while(true){ - //detect row mismatch. break loop if mismatch - int currentNodeDepth = rowLength; - int rowTokenComparison = compareToCurrentToken(key); - if(rowTokenComparison != 0){ - return fixRowTokenMissReverse(rowTokenComparison); - } - - //exact row found, move on to qualifier & ts - if(rowMatchesAfterCurrentPosition(key)){ - return positionAtQualifierTimestamp(key, true); - } - - //detect dead end (no fan to descend into) - if(!currentRowNode.hasFan()){ - if(hasOccurrences()){//must be leaf or nub - populateLastNonRowFields(); - return CellScannerPosition.BEFORE; - }else{ - //TODO i don't think this case is exercised by any tests - return fixRowFanMissReverse(0); - } - } - - //keep hunting for the rest of the row - byte searchForByte = PrivateCellUtil.getRowByte(key, currentNodeDepth); - fanIndex = currentRowNode.whichFanNode(searchForByte); - if(fanIndex < 0){//no matching row. return early - int insertionPoint = -fanIndex - 1; - return fixRowFanMissReverse(insertionPoint); - } - //found a match, so dig deeper into the tree - followFan(fanIndex); - } - } - - /** - * Identical workflow as positionAtOrBefore, but split them to avoid having ~10 extra - * if-statements. Priority on readability and debugability. - */ - @Override - public CellScannerPosition positionAtOrAfter(Cell key) { - reInitFirstNode(); - int fanIndex = -1; - - while(true){ - //detect row mismatch. break loop if mismatch - int currentNodeDepth = rowLength; - int rowTokenComparison = compareToCurrentToken(key); - if(rowTokenComparison != 0){ - return fixRowTokenMissForward(rowTokenComparison); - } - - //exact row found, move on to qualifier & ts - if(rowMatchesAfterCurrentPosition(key)){ - return positionAtQualifierTimestamp(key, false); - } - - //detect dead end (no fan to descend into) - if(!currentRowNode.hasFan()){ - if(hasOccurrences()){ - if (rowLength < key.getRowLength()) { - nextRow(); - } else { - populateFirstNonRowFields(); - } - return CellScannerPosition.AFTER; - }else{ - //TODO i don't think this case is exercised by any tests - return fixRowFanMissForward(0); - } - } - - //keep hunting for the rest of the row - byte searchForByte = PrivateCellUtil.getRowByte(key, currentNodeDepth); - fanIndex = currentRowNode.whichFanNode(searchForByte); - if(fanIndex < 0){//no matching row. return early - int insertionPoint = -fanIndex - 1; - return fixRowFanMissForward(insertionPoint); - } - //found a match, so dig deeper into the tree - followFan(fanIndex); - } - } - - @Override - public boolean seekForwardTo(Cell key) { - if(currentPositionIsAfter(key)){ - //our position is after the requested key, so can't do anything - return false; - } - return positionAt(key); - } - - @Override - public CellScannerPosition seekForwardToOrBefore(Cell key) { - //Do we even need this check or should upper layers avoid this situation. It's relatively - //expensive compared to the rest of the seek operation. - if(currentPositionIsAfter(key)){ - //our position is after the requested key, so can't do anything - return CellScannerPosition.AFTER; - } - - return positionAtOrBefore(key); - } - - @Override - public CellScannerPosition seekForwardToOrAfter(Cell key) { - //Do we even need this check or should upper layers avoid this situation. It's relatively - //expensive compared to the rest of the seek operation. - if(currentPositionIsAfter(key)){ - //our position is after the requested key, so can't do anything - return CellScannerPosition.AFTER; - } - - return positionAtOrAfter(key); - } - - /** - * The content of the buffers doesn't matter here, only that afterLast=true and beforeFirst=false - */ - @Override - public void positionAfterLastCell() { - resetToBeforeFirstEntry(); - beforeFirst = false; - afterLast = true; - } - - - /***************** Object methods ***************************/ - - @Override - public boolean equals(Object obj) { - //trivial override to confirm intent (findbugs) - return super.equals(obj); - } - - - /****************** internal methods ************************/ - - protected boolean currentPositionIsAfter(Cell cell){ - return compareTo(cell) > 0; - } - - protected CellScannerPosition positionAtQualifierTimestamp(Cell key, boolean beforeOnMiss) { - int minIndex = 0; - int maxIndex = currentRowNode.getLastCellIndex(); - int diff; - while (true) { - int midIndex = (maxIndex + minIndex) / 2;//don't worry about overflow - diff = populateNonRowFieldsAndCompareTo(midIndex, key); - - if (diff == 0) {// found exact match - return CellScannerPosition.AT; - } else if (minIndex == maxIndex) {// even termination case - break; - } else if ((minIndex + 1) == maxIndex) {// odd termination case - diff = populateNonRowFieldsAndCompareTo(maxIndex, key); - if(diff > 0){ - diff = populateNonRowFieldsAndCompareTo(minIndex, key); - } - break; - } else if (diff < 0) {// keep going forward - minIndex = currentCellIndex; - } else {// went past it, back up - maxIndex = currentCellIndex; - } - } - - if (diff == 0) { - return CellScannerPosition.AT; - - } else if (diff < 0) {// we are before key - if (beforeOnMiss) { - return CellScannerPosition.BEFORE; - } - if (advance()) { - return CellScannerPosition.AFTER; - } - return CellScannerPosition.AFTER_LAST; - - } else {// we are after key - if (!beforeOnMiss) { - return CellScannerPosition.AFTER; - } - if (previous()) { - return CellScannerPosition.BEFORE; - } - return CellScannerPosition.BEFORE_FIRST; - } - } - - /** - * compare this.row to key.row but starting at the current rowLength - * @param key Cell being searched for - * @return true if row buffer contents match key.row - */ - protected boolean rowMatchesAfterCurrentPosition(Cell key) { - if (!currentRowNode.hasOccurrences()) { - return false; - } - int thatRowLength = key.getRowLength(); - if (rowLength != thatRowLength) { - return false; - } - return true; - } - - // TODO move part of this to Cell comparator? - /** - * Compare only the bytes within the window of the current token - * @param key - * @return return -1 if key is lessThan (before) this, 0 if equal, and 1 if key is after - */ - protected int compareToCurrentToken(Cell key) { - int startIndex = rowLength - currentRowNode.getTokenLength(); - int endIndexExclusive = startIndex + currentRowNode.getTokenLength(); - for (int i = startIndex; i < endIndexExclusive; ++i) { - if (i >= key.getRowLength()) {// key was shorter, so it's first - return -1; - } - byte keyByte = PrivateCellUtil.getRowByte(key, i); - byte thisByte = rowBuffer[i]; - if (keyByte == thisByte) { - continue; - } - return UnsignedBytes.compare(keyByte, thisByte); - } - if (!currentRowNode.hasOccurrences() && rowLength >= key.getRowLength()) { // key was shorter - return -1; - } - return 0; - } - - protected void followLastFansUntilExhausted(){ - while(currentRowNode.hasFan()){ - followLastFan(); - } - } - - - /****************** complete seek when token mismatch ******************/ - - /** - * @param searcherIsAfterInputKey <0: input key is before the searcher's position<br> - * >0: input key is after the searcher's position - */ - protected CellScannerPosition fixRowTokenMissReverse(int searcherIsAfterInputKey) { - if (searcherIsAfterInputKey < 0) {//searcher position is after the input key, so back up - boolean foundPreviousRow = previousRow(true); - if(foundPreviousRow){ - populateLastNonRowFields(); - return CellScannerPosition.BEFORE; - }else{ - return CellScannerPosition.BEFORE_FIRST; - } - - }else{//searcher position is before the input key - if(currentRowNode.hasOccurrences()){ - populateFirstNonRowFields(); - return CellScannerPosition.BEFORE; - } - boolean foundNextRow = nextRow(); - if(foundNextRow){ - return CellScannerPosition.AFTER; - }else{ - return CellScannerPosition.AFTER_LAST; - } - } - } - - /** - * @param searcherIsAfterInputKey <0: input key is before the searcher's position<br> - * >0: input key is after the searcher's position - */ - protected CellScannerPosition fixRowTokenMissForward(int searcherIsAfterInputKey) { - if (searcherIsAfterInputKey < 0) {//searcher position is after the input key - if(currentRowNode.hasOccurrences()){ - populateFirstNonRowFields(); - return CellScannerPosition.AFTER; - } - boolean foundNextRow = nextRow(); - if(foundNextRow){ - return CellScannerPosition.AFTER; - }else{ - return CellScannerPosition.AFTER_LAST; - } - - }else{//searcher position is before the input key, so go forward - discardCurrentRowNode(true); - boolean foundNextRow = nextRow(); - if(foundNextRow){ - return CellScannerPosition.AFTER; - }else{ - return CellScannerPosition.AFTER_LAST; - } - } - } - - - /****************** complete seek when fan mismatch ******************/ - - protected CellScannerPosition fixRowFanMissReverse(int fanInsertionPoint){ - if(fanInsertionPoint == 0){//we need to back up a row - if (currentRowNode.hasOccurrences()) { - populateLastNonRowFields(); - return CellScannerPosition.BEFORE; - } - boolean foundPreviousRow = previousRow(true);//true -> position on last cell in row - if(foundPreviousRow){ - populateLastNonRowFields(); - return CellScannerPosition.BEFORE; - } - return CellScannerPosition.BEFORE_FIRST; - } - - //follow the previous fan, but then descend recursively forward - followFan(fanInsertionPoint - 1); - followLastFansUntilExhausted(); - populateLastNonRowFields(); - return CellScannerPosition.BEFORE; - } - - protected CellScannerPosition fixRowFanMissForward(int fanInsertionPoint){ - if(fanInsertionPoint >= currentRowNode.getFanOut()){ - discardCurrentRowNode(true); - if (!nextRow()) { - return CellScannerPosition.AFTER_LAST; - } else { - return CellScannerPosition.AFTER; - } - } - - followFan(fanInsertionPoint); - if(hasOccurrences()){ - populateFirstNonRowFields(); - return CellScannerPosition.AFTER; - } - - if(nextRowInternal()){ - populateFirstNonRowFields(); - return CellScannerPosition.AFTER; - - }else{ - return CellScannerPosition.AFTER_LAST; - } - } - -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java deleted file mode 100644 index 198ded0..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java +++ /dev/null @@ -1,311 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode; - - -import java.nio.ByteBuffer; -import org.apache.hadoop.hbase.ByteBufferCell; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.PrivateCellUtil; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.SettableSequenceId; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.nio.ByteBuff; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ObjectIntPair; - -/** - * As the PrefixTreeArrayScanner moves through the tree bytes, it changes the - * values in the fields of this class so that Cell logic can be applied, but - * without allocating new memory for every Cell iterated through. - */ -@InterfaceAudience.Private -public class PrefixTreeCell extends ByteBufferCell implements SettableSequenceId, - Comparable<Cell> { - // Create a reference here? Can be removed too - protected CellComparator comparator = CellComparatorImpl.COMPARATOR; - - /********************** static **********************/ - - public static final KeyValue.Type[] TYPES = new KeyValue.Type[256]; - static { - for (KeyValue.Type type : KeyValue.Type.values()) { - TYPES[type.getCode() & 0xff] = type; - } - } - - // Same as KeyValue constructor. Only used to avoid NPE's when full cell - // hasn't been initialized. - public static final KeyValue.Type DEFAULT_TYPE = KeyValue.Type.Put; - - /******************** fields ************************/ - - protected ByteBuff block; - // we could also avoid setting the mvccVersion in the scanner/searcher, but - // this is simpler - protected boolean includeMvccVersion; - - protected byte[] rowBuffer; - protected int rowLength; - - protected byte[] familyBuffer; - protected int familyOffset; - protected int familyLength; - - protected byte[] qualifierBuffer;// aligned to the end of the array - protected int qualifierOffset; - protected int qualifierLength; - - protected Long timestamp; - protected Long mvccVersion; - - protected KeyValue.Type type; - - protected int absoluteValueOffset; - protected int valueLength; - - protected byte[] tagsBuffer; - protected int tagsOffset; - protected int tagsLength; - // Pair to set the value ByteBuffer and its offset - protected ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<>(); - - /********************** Cell methods ******************/ - - /** - * For debugging. Currently creates new KeyValue to utilize its toString() - * method. - */ - @Override - public String toString() { - return getKeyValueString(); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof Cell)) { - return false; - } - // Temporary hack to maintain backwards compatibility with KeyValue.equals - return PrivateCellUtil.equalsIgnoreMvccVersion(this, (Cell) obj); - - // TODO return CellComparator.equals(this, (Cell)obj);//see HBASE-6907 - } - - @Override - public int hashCode() { - return calculateHashForKey(this); - } - - private int calculateHashForKey(Cell cell) { - // pre-calculate the 3 hashes made of byte ranges - int rowHash = Bytes.hashCode(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); - int familyHash = Bytes.hashCode(cell.getFamilyArray(), cell.getFamilyOffset(), - cell.getFamilyLength()); - int qualifierHash = Bytes.hashCode(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()); - - // combine the 6 sub-hashes - int hash = 31 * rowHash + familyHash; - hash = 31 * hash + qualifierHash; - hash = 31 * hash + (int) cell.getTimestamp(); - hash = 31 * hash + cell.getTypeByte(); - return hash; - } - - @Override - public int compareTo(Cell other) { - return comparator.compare(this, other); - } - - @Override - public long getTimestamp() { - return timestamp; - } - - @Override - public long getSequenceId() { - if (!includeMvccVersion) { - return 0L; - } - return mvccVersion; - } - - @Override - public int getValueLength() { - return valueLength; - } - - @Override - public byte[] getRowArray() { - return rowBuffer; - } - - @Override - public int getRowOffset() { - return 0; - } - - @Override - public short getRowLength() { - return (short) rowLength; - } - - @Override - public byte[] getFamilyArray() { - return familyBuffer; - } - - @Override - public int getFamilyOffset() { - return familyOffset; - } - - @Override - public byte getFamilyLength() { - return (byte) familyLength; - } - - @Override - public byte[] getQualifierArray() { - return qualifierBuffer; - } - - @Override - public int getQualifierOffset() { - return qualifierOffset; - } - - @Override - public int getQualifierLength() { - return qualifierLength; - } - - @Override - public byte[] getValueArray() { - if (this.pair.getFirst().hasArray()) { - return this.pair.getFirst().array(); - } else { - // Just in case getValueArray is called on offheap BB - byte[] val = new byte[valueLength]; - ByteBufferUtils.copyFromBufferToArray(val, this.pair.getFirst(), this.pair.getSecond(), 0, - valueLength); - return val; - } - } - - @Override - public int getValueOffset() { - if (this.pair.getFirst().hasArray()) { - return this.pair.getSecond() + this.pair.getFirst().arrayOffset(); - } else { - return 0; - } - } - - @Override - public byte getTypeByte() { - return type.getCode(); - } - - /************************* helper methods *************************/ - - /** - * Need this separate method so we can call it from subclasses' toString() - * methods - */ - protected String getKeyValueString() { - KeyValue kv = KeyValueUtil.copyToNewKeyValue(this); - return kv.toString(); - } - - @Override - public int getTagsOffset() { - return tagsOffset; - } - - @Override - public int getTagsLength() { - return tagsLength; - } - - @Override - public byte[] getTagsArray() { - return this.tagsBuffer; - } - - @Override - public void setSequenceId(long seqId) { - mvccVersion = seqId; - } - - @Override - public ByteBuffer getRowByteBuffer() { - return ByteBuffer.wrap(rowBuffer); - } - - @Override - public int getRowPosition() { - return 0; - } - - @Override - public ByteBuffer getFamilyByteBuffer() { - return ByteBuffer.wrap(familyBuffer); - } - - @Override - public int getFamilyPosition() { - return getFamilyOffset(); - } - - @Override - public ByteBuffer getQualifierByteBuffer() { - return ByteBuffer.wrap(qualifierBuffer); - } - - @Override - public int getQualifierPosition() { - return getQualifierOffset(); - } - - @Override - public ByteBuffer getValueByteBuffer() { - return pair.getFirst(); - } - - @Override - public int getValuePosition() { - return pair.getSecond(); - } - - @Override - public ByteBuffer getTagsByteBuffer() { - return ByteBuffer.wrap(tagsBuffer); - } - - @Override - public int getTagsPosition() { - return getTagsOffset(); - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java deleted file mode 100644 index a684819..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode.column; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.codec.prefixtree.encode.other.ColumnNodeType; -import org.apache.hadoop.hbase.nio.ByteBuff; -import org.apache.hadoop.hbase.util.vint.UFIntTool; -import org.apache.hadoop.hbase.util.vint.UVIntTool; - -@InterfaceAudience.Private -public class ColumnNodeReader { - - /**************** fields ************************/ - - protected PrefixTreeBlockMeta blockMeta; - protected ByteBuff block; - protected ColumnNodeType nodeType; - protected byte[] columnBuffer; - - protected int offsetIntoBlock; - - protected int tokenOffsetIntoBlock; - protected int tokenLength; - protected int parentStartPosition; - - - /************** construct *************************/ - - public ColumnNodeReader(byte[] columnBuffer, ColumnNodeType nodeType) { - this.columnBuffer = columnBuffer; - this.nodeType = nodeType; - } - - public void initOnBlock(PrefixTreeBlockMeta blockMeta, ByteBuff block) { - this.blockMeta = blockMeta; - this.block = block; - } - - - /************* methods *****************************/ - - public void positionAt(int offsetIntoBlock) { - this.offsetIntoBlock = offsetIntoBlock; - tokenLength = UVIntTool.getInt(block, offsetIntoBlock); - tokenOffsetIntoBlock = offsetIntoBlock + UVIntTool.numBytes(tokenLength); - int parentStartPositionIndex = tokenOffsetIntoBlock + tokenLength; - int offsetWidth; - if(nodeType == ColumnNodeType.FAMILY) { - offsetWidth = blockMeta.getFamilyOffsetWidth(); - } else if(nodeType == ColumnNodeType.QUALIFIER) { - offsetWidth = blockMeta.getQualifierOffsetWidth(); - } else { - offsetWidth = blockMeta.getTagsOffsetWidth(); - } - parentStartPosition = (int) UFIntTool.fromBytes(block, parentStartPositionIndex, offsetWidth); - } - - public void prependTokenToBuffer(int bufferStartIndex) { - block.get(tokenOffsetIntoBlock, columnBuffer, bufferStartIndex, tokenLength); - } - - public boolean isRoot() { - if (nodeType == ColumnNodeType.FAMILY) { - return offsetIntoBlock == blockMeta.getAbsoluteFamilyOffset(); - } else if (nodeType == ColumnNodeType.QUALIFIER) { - return offsetIntoBlock == blockMeta.getAbsoluteQualifierOffset(); - } else { - return offsetIntoBlock == blockMeta.getAbsoluteTagsOffset(); - } - } - - - /************** standard methods *********************/ - - @Override - public String toString() { - return super.toString() + "[" + offsetIntoBlock + "]"; - } - - - /****************** get/set ****************************/ - - public int getTokenLength() { - return tokenLength; - } - - public int getParentStartPosition() { - return parentStartPosition; - } - -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java deleted file mode 100644 index 87b2d0e..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode.column; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.codec.prefixtree.encode.other.ColumnNodeType; -import org.apache.hadoop.hbase.nio.ByteBuff; - -/** - * Position one of these appropriately in the data block and you can call its methods to retrieve - * the family or qualifier at the current position. - */ -@InterfaceAudience.Private -public class ColumnReader { - - /****************** fields *************************/ - - protected PrefixTreeBlockMeta blockMeta; - - protected byte[] columnBuffer; - protected int columnOffset; - protected int columnLength; - protected ColumnNodeType nodeType; - - protected ColumnNodeReader columnNodeReader; - - - /******************** construct *******************/ - - public ColumnReader(byte[] columnBuffer, ColumnNodeType nodeType) { - this.columnBuffer = columnBuffer; - this.nodeType = nodeType; - this.columnNodeReader = new ColumnNodeReader(columnBuffer, nodeType); - } - - public void initOnBlock(PrefixTreeBlockMeta blockMeta, ByteBuff block) { - this.blockMeta = blockMeta; - clearColumnBuffer(); - columnNodeReader.initOnBlock(blockMeta, block); - } - - - /********************* methods *******************/ - - public ColumnReader populateBuffer(int offsetIntoColumnData) { - clearColumnBuffer(); - int nextRelativeOffset = offsetIntoColumnData; - while (true) { - int absoluteOffset = 0; - if (nodeType == ColumnNodeType.FAMILY) { - absoluteOffset = blockMeta.getAbsoluteFamilyOffset() + nextRelativeOffset; - } else if (nodeType == ColumnNodeType.QUALIFIER) { - absoluteOffset = blockMeta.getAbsoluteQualifierOffset() + nextRelativeOffset; - } else { - absoluteOffset = blockMeta.getAbsoluteTagsOffset() + nextRelativeOffset; - } - columnNodeReader.positionAt(absoluteOffset); - columnOffset -= columnNodeReader.getTokenLength(); - columnLength += columnNodeReader.getTokenLength(); - columnNodeReader.prependTokenToBuffer(columnOffset); - if (columnNodeReader.isRoot()) { - return this; - } - nextRelativeOffset = columnNodeReader.getParentStartPosition(); - } - } - - public byte[] copyBufferToNewArray() {// for testing - byte[] out = new byte[columnLength]; - System.arraycopy(columnBuffer, columnOffset, out, 0, out.length); - return out; - } - - public int getColumnLength() { - return columnLength; - } - - public void clearColumnBuffer() { - columnOffset = columnBuffer.length; - columnLength = 0; - } - - - /****************************** get/set *************************************/ - - public int getColumnOffset() { - return columnOffset; - } - -} - http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java deleted file mode 100644 index 31b52e9..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java +++ /dev/null @@ -1,281 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode.row; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.nio.ByteBuff; -import org.apache.hadoop.hbase.util.vint.UFIntTool; -import org.apache.hadoop.hbase.util.vint.UVIntTool; - -/** - * Position one of these appropriately in the data block and you can call its methods to retrieve - * information necessary to decode the cells in the row. - */ -@InterfaceAudience.Private -public class RowNodeReader { - - /************* fields ***********************************/ - - protected ByteBuff block; - protected int offset; - protected int fanIndex; - - protected int numCells; - - protected int tokenOffset; - protected int tokenLength; - protected int fanOffset; - protected int fanOut; - - protected int familyOffsetsOffset; - protected int qualifierOffsetsOffset; - protected int timestampIndexesOffset; - protected int mvccVersionIndexesOffset; - protected int operationTypesOffset; - protected int valueOffsetsOffset; - protected int valueLengthsOffset; - protected int tagOffsetsOffset; - protected int nextNodeOffsetsOffset; - - - /******************* construct **************************/ - - public void initOnBlock(PrefixTreeBlockMeta blockMeta, ByteBuff block, int offset) { - this.block = block; - - this.offset = offset; - resetFanIndex(); - - this.tokenLength = UVIntTool.getInt(block, offset); - this.tokenOffset = offset + UVIntTool.numBytes(tokenLength); - - this.fanOut = UVIntTool.getInt(block, tokenOffset + tokenLength); - this.fanOffset = tokenOffset + tokenLength + UVIntTool.numBytes(fanOut); - - this.numCells = UVIntTool.getInt(block, fanOffset + fanOut); - - this.familyOffsetsOffset = fanOffset + fanOut + UVIntTool.numBytes(numCells); - this.qualifierOffsetsOffset = familyOffsetsOffset + numCells * blockMeta.getFamilyOffsetWidth(); - this.tagOffsetsOffset = this.qualifierOffsetsOffset + numCells * blockMeta.getQualifierOffsetWidth(); - // TODO : This code may not be needed now..As we always consider tags to be present - if(blockMeta.getTagsOffsetWidth() == 0) { - // Make both of them same so that we know that there are no tags - this.tagOffsetsOffset = this.qualifierOffsetsOffset; - this.timestampIndexesOffset = qualifierOffsetsOffset + numCells * blockMeta.getQualifierOffsetWidth(); - } else { - this.timestampIndexesOffset = tagOffsetsOffset + numCells * blockMeta.getTagsOffsetWidth(); - } - this.mvccVersionIndexesOffset = timestampIndexesOffset + numCells - * blockMeta.getTimestampIndexWidth(); - this.operationTypesOffset = mvccVersionIndexesOffset + numCells - * blockMeta.getMvccVersionIndexWidth(); - this.valueOffsetsOffset = operationTypesOffset + numCells * blockMeta.getKeyValueTypeWidth(); - this.valueLengthsOffset = valueOffsetsOffset + numCells * blockMeta.getValueOffsetWidth(); - this.nextNodeOffsetsOffset = valueLengthsOffset + numCells * blockMeta.getValueLengthWidth(); - } - - - /******************** methods ****************************/ - - public boolean isLeaf() { - return fanOut == 0; - } - - public boolean isNub() { - return fanOut > 0 && numCells > 0; - } - - public boolean isBranch() { - return fanOut > 0 && numCells == 0; - } - - public boolean hasOccurrences() { - return numCells > 0; - } - - public int getTokenArrayOffset(){ - return tokenOffset; - } - - public int getTokenLength() { - return tokenLength; - } - - public byte getFanByte(int i) { - return block.get(fanOffset + i); - } - - /** - * for debugging - */ - protected String getFanByteReadable(int i){ - return ByteBuff.toStringBinary(block, fanOffset + i, 1); - } - - public int getFamilyOffset(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getFamilyOffsetWidth(); - int startIndex = familyOffsetsOffset + fIntWidth * index; - return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - } - - public int getColumnOffset(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getQualifierOffsetWidth(); - int startIndex = qualifierOffsetsOffset + fIntWidth * index; - return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - } - - public int getTagOffset(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getTagsOffsetWidth(); - int startIndex = tagOffsetsOffset + fIntWidth * index; - return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - } - - public int getTimestampIndex(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getTimestampIndexWidth(); - int startIndex = timestampIndexesOffset + fIntWidth * index; - return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - } - - public int getMvccVersionIndex(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getMvccVersionIndexWidth(); - int startIndex = mvccVersionIndexesOffset + fIntWidth * index; - return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - } - - public int getType(int index, PrefixTreeBlockMeta blockMeta) { - if (blockMeta.isAllSameType()) { - return blockMeta.getAllTypes(); - } - return block.get(operationTypesOffset + index); - } - - public int getValueOffset(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getValueOffsetWidth(); - int startIndex = valueOffsetsOffset + fIntWidth * index; - int offset = (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - return offset; - } - - public int getValueLength(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getValueLengthWidth(); - int startIndex = valueLengthsOffset + fIntWidth * index; - int length = (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - return length; - } - - public int getNextNodeOffset(int index, PrefixTreeBlockMeta blockMeta) { - int fIntWidth = blockMeta.getNextNodeOffsetWidth(); - int startIndex = nextNodeOffsetsOffset + fIntWidth * index; - return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth); - } - - public String getBranchNubLeafIndicator() { - if (isNub()) { - return "N"; - } - return isBranch() ? "B" : "L"; - } - - public boolean hasChildren() { - return fanOut > 0; - } - - public int getLastFanIndex() { - return fanOut - 1; - } - - public int getLastCellIndex() { - return numCells - 1; - } - - public int getNumCells() { - return numCells; - } - - public int getFanOut() { - return fanOut; - } - - public byte[] getToken() { - byte[] newToken = new byte[tokenLength]; - block.get(tokenOffset, newToken, 0, tokenLength); - return newToken; - } - - public int getOffset() { - return offset; - } - - public int whichFanNode(byte searchForByte) { - if( ! hasFan()){ - throw new IllegalStateException("This row node has no fan, so can't search it"); - } - int fanIndexInBlock = ByteBuff.unsignedBinarySearch(block, fanOffset, fanOffset + fanOut, - searchForByte); - if (fanIndexInBlock >= 0) {// found it, but need to adjust for position of fan in overall block - return fanIndexInBlock - fanOffset; - } - return fanIndexInBlock + fanOffset;// didn't find it, so compensate in reverse - } - - public void resetFanIndex() { - fanIndex = -1;// just the way the logic currently works - } - - public int getFanIndex() { - return fanIndex; - } - - public void setFanIndex(int fanIndex) { - this.fanIndex = fanIndex; - } - - public boolean hasFan(){ - return fanOut > 0; - } - - public boolean hasPreviousFanNodes() { - return fanOut > 0 && fanIndex > 0; - } - - public boolean hasMoreFanNodes() { - return fanIndex < getLastFanIndex(); - } - - public boolean isOnLastFanNode() { - return !hasMoreFanNodes(); - } - - - /*************** standard methods **************************/ - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("fan:" + ByteBuff.toStringBinary(block, fanOffset, fanOut)); - sb.append(",token:" + ByteBuff.toStringBinary(block, tokenOffset, tokenLength)); - sb.append(",numCells:" + numCells); - sb.append(",fanIndex:"+fanIndex); - if(fanIndex>=0){ - sb.append("("+getFanByteReadable(fanIndex)+")"); - } - return sb.toString(); - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java deleted file mode 100644 index fe3ad12..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode.timestamp; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.nio.ByteBuff; -import org.apache.hadoop.hbase.util.vint.UFIntTool; - -/** - * Given a block and its blockMeta, this will decode the MvccVersion for the i-th Cell in the block. - */ -@InterfaceAudience.Private -public class MvccVersionDecoder { - - protected PrefixTreeBlockMeta blockMeta; - protected ByteBuff block; - - - /************** construct ***********************/ - - public MvccVersionDecoder() { - } - - public void initOnBlock(PrefixTreeBlockMeta blockMeta, ByteBuff block) { - this.block = block; - this.blockMeta = blockMeta; - } - - - /************** methods *************************/ - - public long getMvccVersion(int index) { - if (blockMeta.getMvccVersionIndexWidth() == 0) {//all mvccVersions in the block were identical - return blockMeta.getMinMvccVersion(); - } - int startIndex = blockMeta.getAbsoluteMvccVersionOffset() - + blockMeta.getMvccVersionDeltaWidth() * index; - long delta = UFIntTool.fromBytes(block, startIndex, blockMeta.getMvccVersionDeltaWidth()); - return blockMeta.getMinMvccVersion() + delta; - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java deleted file mode 100644 index 063d0de..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.decode.timestamp; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta; -import org.apache.hadoop.hbase.nio.ByteBuff; -import org.apache.hadoop.hbase.util.vint.UFIntTool; - -/** - * Given a block and its blockMeta, this will decode the timestamp for the i-th Cell in the block. - */ -@InterfaceAudience.Private -public class TimestampDecoder { - - protected PrefixTreeBlockMeta blockMeta; - protected ByteBuff block; - - - /************** construct ***********************/ - - public TimestampDecoder() { - } - - public void initOnBlock(PrefixTreeBlockMeta blockMeta, ByteBuff block) { - this.block = block; - this.blockMeta = blockMeta; - } - - - /************** methods *************************/ - - public long getLong(int index) { - if (blockMeta.getTimestampIndexWidth() == 0) {//all timestamps in the block were identical - return blockMeta.getMinTimestamp(); - } - int startIndex = blockMeta.getAbsoluteTimestampOffset() + blockMeta.getTimestampDeltaWidth() - * index; - long delta = UFIntTool.fromBytes(block, startIndex, blockMeta.getTimestampDeltaWidth()); - return blockMeta.getMinTimestamp() + delta; - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java deleted file mode 100644 index b73d27a..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.encode; - -import java.io.OutputStream; - -import org.apache.yetus.audience.InterfaceAudience; - -/** - * Retrieve PrefixTreeEncoders from this factory which handles pooling them and preparing the - * ones retrieved from the pool for usage. - */ -@InterfaceAudience.Private -public class EncoderFactory { - - private static final EncoderPool POOL = new EncoderPoolImpl(); - - - public static PrefixTreeEncoder checkOut(OutputStream outputStream, boolean includeMvccVersion) { - return POOL.checkOut(outputStream, includeMvccVersion); - } - - public static void checkIn(PrefixTreeEncoder encoder) { - POOL.checkIn(encoder); - } - - - /**************************** helper ******************************/ - - protected static PrefixTreeEncoder prepareEncoder(PrefixTreeEncoder encoder, - OutputStream outputStream, boolean includeMvccVersion) { - PrefixTreeEncoder ret = encoder; - if (encoder == null) { - ret = new PrefixTreeEncoder(outputStream, includeMvccVersion); - } - ret.reset(outputStream, includeMvccVersion); - return ret; - } - -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java deleted file mode 100644 index fac75ac..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.encode; - -import java.io.OutputStream; - -import org.apache.yetus.audience.InterfaceAudience; - - -@InterfaceAudience.Private -public interface EncoderPool { - - PrefixTreeEncoder checkOut(OutputStream outputStream, boolean includeMvccVersion); - void checkIn(PrefixTreeEncoder encoder); - -} http://git-wip-us.apache.org/repos/asf/hbase/blob/f8c58930/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPoolImpl.java ---------------------------------------------------------------------- diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPoolImpl.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPoolImpl.java deleted file mode 100644 index 0d8a6d0..0000000 --- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPoolImpl.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.hbase.codec.prefixtree.encode; - -import java.io.OutputStream; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - -import org.apache.yetus.audience.InterfaceAudience; - -@InterfaceAudience.Private -public class EncoderPoolImpl implements EncoderPool { - - private BlockingQueue<PrefixTreeEncoder> unusedEncoders = new LinkedBlockingQueue<>(); - - @Override - public PrefixTreeEncoder checkOut(OutputStream outputStream, boolean includeMvccVersion) { - PrefixTreeEncoder encoder = unusedEncoders.poll(); - if (encoder == null) { - encoder = new PrefixTreeEncoder(outputStream, includeMvccVersion); - } else { - encoder.reset(outputStream, includeMvccVersion); - } - return encoder; - } - - @Override - public void checkIn(PrefixTreeEncoder encoder) { - this.unusedEncoders.add(encoder); - } -}