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}