001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.regionserver; 020 021import java.io.IOException; 022import java.nio.ByteBuffer; 023import org.apache.hadoop.hbase.ByteBufferKeyValue; 024import org.apache.hadoop.hbase.Cell; 025import org.apache.hadoop.hbase.CellComparator; 026import org.apache.hadoop.hbase.CellUtil; 027import org.apache.hadoop.hbase.ExtendedCell; 028import org.apache.hadoop.hbase.KeyValue; 029import org.apache.hadoop.hbase.util.ByteBufferUtils; 030import org.apache.hadoop.hbase.util.ClassSize; 031import org.apache.yetus.audience.InterfaceAudience; 032 033 034/** 035 * CellChunkImmutableSegment extends the API supported by a {@link Segment}, 036 * and {@link ImmutableSegment}. This immutable segment is working with CellSet with 037 * CellChunkMap delegatee. 038 */ 039@InterfaceAudience.Private 040public class CellChunkImmutableSegment extends ImmutableSegment { 041 042 public static final long DEEP_OVERHEAD_CCM = 043 ImmutableSegment.DEEP_OVERHEAD + ClassSize.CELL_CHUNK_MAP; 044 public static final float INDEX_CHUNK_UNUSED_SPACE_PRECENTAGE = 0.1f; 045 046 ///////////////////// CONSTRUCTORS ///////////////////// 047 /**------------------------------------------------------------------------ 048 * C-tor to be used when new CellChunkImmutableSegment is built as a result of compaction/merge 049 * of a list of older ImmutableSegments. 050 * The given iterator returns the Cells that "survived" the compaction. 051 */ 052 protected CellChunkImmutableSegment(CellComparator comparator, MemStoreSegmentsIterator iterator, 053 MemStoreLAB memStoreLAB, int numOfCells, MemStoreCompactionStrategy.Action action) { 054 super(null, comparator, memStoreLAB); // initialize the CellSet with NULL 055 long indexOverhead = DEEP_OVERHEAD_CCM; 056 // memStoreLAB cannot be null in this class 057 boolean onHeap = getMemStoreLAB().isOnHeap(); 058 // initiate the heapSize with the size of the segment metadata 059 if (onHeap) { 060 incMemStoreSize(0, indexOverhead, 0, 0); 061 } else { 062 incMemStoreSize(0, 0, indexOverhead, 0); 063 } 064 // build the new CellSet based on CellArrayMap and update the CellSet of the new Segment 065 initializeCellSet(numOfCells, iterator, action); 066 } 067 068 /**------------------------------------------------------------------------ 069 * C-tor to be used when new CellChunkImmutableSegment is built as a result of flattening 070 * of CSLMImmutableSegment 071 * The given iterator returns the Cells that "survived" the compaction. 072 */ 073 protected CellChunkImmutableSegment(CSLMImmutableSegment segment, 074 MemStoreSizing memstoreSizing, MemStoreCompactionStrategy.Action action) { 075 super(segment); // initiailize the upper class 076 long indexOverhead = -CSLMImmutableSegment.DEEP_OVERHEAD_CSLM + DEEP_OVERHEAD_CCM; 077 // memStoreLAB cannot be null in this class 078 boolean onHeap = getMemStoreLAB().isOnHeap(); 079 // initiate the heapSize with the size of the segment metadata 080 if(onHeap) { 081 incMemStoreSize(0, indexOverhead, 0, 0); 082 memstoreSizing.incMemStoreSize(0, indexOverhead, 0, 0); 083 } else { 084 incMemStoreSize(0, -CSLMImmutableSegment.DEEP_OVERHEAD_CSLM, DEEP_OVERHEAD_CCM, 0); 085 memstoreSizing.incMemStoreSize(0, -CSLMImmutableSegment.DEEP_OVERHEAD_CSLM, DEEP_OVERHEAD_CCM, 086 0); 087 } 088 int numOfCells = segment.getCellsCount(); 089 // build the new CellSet based on CellChunkMap 090 reinitializeCellSet(numOfCells, segment.getScanner(Long.MAX_VALUE), segment.getCellSet(), 091 memstoreSizing, action); 092 // arrange the meta-data size, decrease all meta-data sizes related to SkipList; 093 // add sizes of CellChunkMap entry, decrease also Cell object sizes 094 // (reinitializeCellSet doesn't take the care for the sizes) 095 long newSegmentSizeDelta = numOfCells*(indexEntrySize()-ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY); 096 if(onHeap) { 097 incMemStoreSize(0, newSegmentSizeDelta, 0, 0); 098 memstoreSizing.incMemStoreSize(0, newSegmentSizeDelta, 0, 0); 099 } else { 100 incMemStoreSize(0, 0, newSegmentSizeDelta, 0); 101 memstoreSizing.incMemStoreSize(0, 0, newSegmentSizeDelta, 0); 102 103 } 104 } 105 106 @Override 107 protected long indexEntryOnHeapSize(boolean onHeap) { 108 if(onHeap) { 109 return indexEntrySize(); 110 } 111 // else the index is allocated off-heap 112 return 0; 113 } 114 115 @Override 116 protected long indexEntryOffHeapSize(boolean offHeap) { 117 if(offHeap) { 118 return indexEntrySize(); 119 } 120 // else the index is allocated on-heap 121 return 0; 122 } 123 124 @Override 125 protected long indexEntrySize() { 126 return ((long) ClassSize.CELL_CHUNK_MAP_ENTRY - KeyValue.FIXED_OVERHEAD); 127 } 128 129 @Override 130 protected boolean canBeFlattened() { 131 return false; 132 } 133 134 ///////////////////// PRIVATE METHODS ///////////////////// 135 /*------------------------------------------------------------------------*/ 136 // Create CellSet based on CellChunkMap from compacting iterator 137 private void initializeCellSet(int numOfCells, MemStoreSegmentsIterator iterator, 138 MemStoreCompactionStrategy.Action action) { 139 140 int numOfCellsAfterCompaction = 0; 141 int currentChunkIdx = 0; 142 int offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER; 143 int numUniqueKeys=0; 144 Cell prev = null; 145 Chunk[] chunks = allocIndexChunks(numOfCells); 146 while (iterator.hasNext()) { // the iterator hides the elimination logic for compaction 147 boolean alreadyCopied = false; 148 Cell c = iterator.next(); 149 numOfCellsAfterCompaction++; 150 assert(c instanceof ExtendedCell); 151 if (((ExtendedCell)c).getChunkId() == ExtendedCell.CELL_NOT_BASED_ON_CHUNK) { 152 // CellChunkMap assumes all cells are allocated on MSLAB. 153 // Therefore, cells which are not allocated on MSLAB initially, 154 // are copied into MSLAB here. 155 c = copyCellIntoMSLAB(c, null); //no memstore sizing object to update 156 alreadyCopied = true; 157 } 158 if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunks[currentChunkIdx].size) { 159 currentChunkIdx++; // continue to the next index chunk 160 offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER; 161 } 162 if (action == MemStoreCompactionStrategy.Action.COMPACT && !alreadyCopied) { 163 // for compaction copy cell to the new segment (MSLAB copy) 164 c = maybeCloneWithAllocator(c, false); 165 } 166 offsetInCurentChunk = // add the Cell reference to the index chunk 167 createCellReference((ByteBufferKeyValue)c, chunks[currentChunkIdx].getData(), 168 offsetInCurentChunk); 169 // the sizes still need to be updated in the new segment 170 // second parameter true, because in compaction/merge the addition of the cell to new segment 171 // is always successful 172 updateMetaInfo(c, true, null); // updates the size per cell 173 if(action == MemStoreCompactionStrategy.Action.MERGE_COUNT_UNIQUE_KEYS) { 174 //counting number of unique keys 175 if (prev != null) { 176 if (!CellUtil.matchingRowColumnBytes(prev, c)) { 177 numUniqueKeys++; 178 } 179 } else { 180 numUniqueKeys++; 181 } 182 } 183 prev = c; 184 } 185 if(action == MemStoreCompactionStrategy.Action.COMPACT) { 186 numUniqueKeys = numOfCells; 187 } else if(action != MemStoreCompactionStrategy.Action.MERGE_COUNT_UNIQUE_KEYS) { 188 numUniqueKeys = CellSet.UNKNOWN_NUM_UNIQUES; 189 } 190 // build the immutable CellSet 191 CellChunkMap ccm = 192 new CellChunkMap(getComparator(), chunks, 0, numOfCellsAfterCompaction, false); 193 this.setCellSet(null, new CellSet(ccm, numUniqueKeys)); // update the CellSet of this Segment 194 } 195 196 /*------------------------------------------------------------------------*/ 197 // Create CellSet based on CellChunkMap from current ConcurrentSkipListMap based CellSet 198 // (without compacting iterator) 199 // This is a service for not-flat immutable segments 200 private void reinitializeCellSet( 201 int numOfCells, KeyValueScanner segmentScanner, CellSet oldCellSet, 202 MemStoreSizing memstoreSizing, MemStoreCompactionStrategy.Action action) { 203 Cell curCell; 204 Chunk[] chunks = allocIndexChunks(numOfCells); 205 206 int currentChunkIdx = 0; 207 int offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER; 208 209 int numUniqueKeys=0; 210 Cell prev = null; 211 try { 212 while ((curCell = segmentScanner.next()) != null) { 213 assert(curCell instanceof ExtendedCell); 214 if (((ExtendedCell)curCell).getChunkId() == ExtendedCell.CELL_NOT_BASED_ON_CHUNK) { 215 // CellChunkMap assumes all cells are allocated on MSLAB. 216 // Therefore, cells which are not allocated on MSLAB initially, 217 // are copied into MSLAB here. 218 curCell = copyCellIntoMSLAB(curCell, memstoreSizing); 219 } 220 if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunks[currentChunkIdx].size) { 221 // continue to the next metadata chunk 222 currentChunkIdx++; 223 offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER; 224 } 225 offsetInCurentChunk = 226 createCellReference((ByteBufferKeyValue) curCell, chunks[currentChunkIdx].getData(), 227 offsetInCurentChunk); 228 if(action == MemStoreCompactionStrategy.Action.FLATTEN_COUNT_UNIQUE_KEYS) { 229 //counting number of unique keys 230 if (prev != null) { 231 if (!CellUtil.matchingRowColumn(prev, curCell)) { 232 numUniqueKeys++; 233 } 234 } else { 235 numUniqueKeys++; 236 } 237 } 238 prev = curCell; 239 } 240 if(action != MemStoreCompactionStrategy.Action.FLATTEN_COUNT_UNIQUE_KEYS) { 241 numUniqueKeys = CellSet.UNKNOWN_NUM_UNIQUES; 242 } 243 } catch (IOException ie) { 244 throw new IllegalStateException(ie); 245 } finally { 246 segmentScanner.close(); 247 } 248 249 CellChunkMap ccm = new CellChunkMap(getComparator(), chunks, 0, numOfCells, false); 250 // update the CellSet of this Segment 251 this.setCellSet(oldCellSet, new CellSet(ccm, numUniqueKeys)); 252 } 253 254 /*------------------------------------------------------------------------*/ 255 // for a given cell, write the cell representation on the index chunk 256 private int createCellReference(ByteBufferKeyValue cell, ByteBuffer idxBuffer, int idxOffset) { 257 int offset = idxOffset; 258 int dataChunkID = cell.getChunkId(); 259 260 offset = ByteBufferUtils.putInt(idxBuffer, offset, dataChunkID); // write data chunk id 261 offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getOffset()); // offset 262 offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getSerializedSize()); // length 263 offset = ByteBufferUtils.putLong(idxBuffer, offset, cell.getSequenceId()); // seqId 264 265 return offset; 266 } 267 268 private int calculateNumberOfChunks(int numOfCells, int chunkSize) { 269 int numOfCellsInChunk = calcNumOfCellsInChunk(chunkSize); 270 int numberOfChunks = numOfCells / numOfCellsInChunk; 271 if(numOfCells % numOfCellsInChunk != 0) { // if cells cannot be divided evenly between chunks 272 numberOfChunks++; // add one additional chunk 273 } 274 return numberOfChunks; 275 } 276 277 // Assuming we are going to use regular data chunks as index chunks, 278 // we check here how much free space will remain in the last allocated chunk 279 // (the least occupied one). 280 // If the percentage of its remaining free space is above the INDEX_CHUNK_UNUSED_SPACE 281 // threshold, then we will use index chunks (which are smaller) instead. 282 private ChunkCreator.ChunkType useIndexChunks(int numOfCells) { 283 int dataChunkSize = ChunkCreator.getInstance().getChunkSize(); 284 int numOfCellsInChunk = calcNumOfCellsInChunk(dataChunkSize); 285 int cellsInLastChunk = numOfCells % numOfCellsInChunk; 286 if (cellsInLastChunk == 0) { // There is no free space in the last chunk and thus, 287 return ChunkCreator.ChunkType.DATA_CHUNK; // no need to use index chunks. 288 } else { 289 int chunkSpace = dataChunkSize - ChunkCreator.SIZEOF_CHUNK_HEADER; 290 int freeSpaceInLastChunk = chunkSpace - cellsInLastChunk * ClassSize.CELL_CHUNK_MAP_ENTRY; 291 if (freeSpaceInLastChunk > INDEX_CHUNK_UNUSED_SPACE_PRECENTAGE * chunkSpace) { 292 return ChunkCreator.ChunkType.INDEX_CHUNK; 293 } 294 return ChunkCreator.ChunkType.DATA_CHUNK; 295 } 296 } 297 298 private int calcNumOfCellsInChunk(int chunkSize) { 299 int chunkSpace = chunkSize - ChunkCreator.SIZEOF_CHUNK_HEADER; 300 int numOfCellsInChunk = chunkSpace / ClassSize.CELL_CHUNK_MAP_ENTRY; 301 return numOfCellsInChunk; 302 } 303 304 private Chunk[] allocIndexChunks(int numOfCells) { 305 // Decide whether to use regular or small chunks and then 306 // calculate how many chunks we will need for index 307 308 ChunkCreator.ChunkType chunkType = useIndexChunks(numOfCells); 309 int chunkSize = ChunkCreator.getInstance().getChunkSize(chunkType); 310 int numberOfChunks = calculateNumberOfChunks(numOfCells, chunkSize); 311 // all index Chunks are allocated from ChunkCreator 312 Chunk[] chunks = new Chunk[numberOfChunks]; 313 // all index Chunks are allocated from ChunkCreator 314 for (int i = 0; i < numberOfChunks; i++) { 315 chunks[i] = this.getMemStoreLAB().getNewExternalChunk(chunkType); 316 } 317 return chunks; 318 } 319 320 private Cell copyCellIntoMSLAB(Cell cell, MemStoreSizing memstoreSizing) { 321 // Take care for a special case when a cell is copied from on-heap to (probably off-heap) MSLAB. 322 // The cell allocated as an on-heap JVM object (byte array) occupies slightly different 323 // amount of memory, than when the cell serialized and allocated on the MSLAB. 324 // Here, we update the heap size of the new segment only for the difference between object and 325 // serialized size. This is a decrease of the size as serialized cell is a bit smaller. 326 // The actual size of the cell is not added yet, and will be added (only in compaction) 327 // in initializeCellSet#updateMetaInfo(). 328 long oldHeapSize = heapSizeChange(cell, true); 329 long oldOffHeapSize = offHeapSizeChange(cell, true); 330 long oldCellSize = getCellLength(cell); 331 cell = maybeCloneWithAllocator(cell, true); 332 long newHeapSize = heapSizeChange(cell, true); 333 long newOffHeapSize = offHeapSizeChange(cell, true); 334 long newCellSize = getCellLength(cell); 335 long heapOverhead = newHeapSize - oldHeapSize; 336 long offHeapOverhead = newOffHeapSize - oldOffHeapSize; 337 incMemStoreSize(newCellSize - oldCellSize, heapOverhead, offHeapOverhead, 0); 338 if(memstoreSizing != null) { 339 memstoreSizing.incMemStoreSize(newCellSize - oldCellSize, heapOverhead, offHeapOverhead, 0); 340 } 341 return cell; 342 } 343}