HBASE-26286: Add support for specifying store file tracker when restoring or cloning...
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / CellChunkMap.java
bloba2c18206271d1088333638e7322aa57d502a66f5
1 /**
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Cellersion 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY CellIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
20 package org.apache.hadoop.hbase.regionserver;
22 import java.nio.ByteBuffer;
24 import org.apache.hadoop.hbase.Cell;
25 import org.apache.yetus.audience.InterfaceAudience;
26 import org.apache.hadoop.hbase.util.Bytes;
27 import org.apache.hadoop.hbase.util.ByteBufferUtils;
28 import org.apache.hadoop.hbase.util.ClassSize;
30 import java.util.Comparator;
33 /**
34 * CellChunkMap is an array of serialized representations of Cell
35 * (pointing to Chunks with full Cell data) and can be allocated both off-heap and on-heap.
37 * CellChunkMap is a byte array (chunk) holding all that is needed to access a Cell, which
38 * is actually saved on another deeper chunk.
39 * Per Cell we have a reference to this deeper byte array B (chunk ID, integer),
40 * offset in bytes in B (integer), length in bytes in B (integer) and seqID of the cell (long).
41 * In order to save reference to byte array we use the Chunk's ID given by ChunkCreator.
43 * The CellChunkMap memory layout on chunk A relevant to a deeper byte array B,
44 * holding the actual cell data:
46 * < header > <--------------- first Cell -----------------> <-- second Cell ...
47 * --------------------------------------------------------------------------------------- ...
48 * integer | integer | integer | integer | long |
49 * 4 bytes | 4 bytes | 4 bytes | 4 bytes | 8 bytes |
50 * ChunkID | chunkID of | offset in B | length of | sequence | ...
51 * of this | chunk B with | where Cell's | Cell's | ID of |
52 * chunk A | Cell data | data starts | data in B | the Cell |
53 * --------------------------------------------------------------------------------------- ...
55 @InterfaceAudience.Private
56 public class CellChunkMap extends CellFlatMap {
58 private final Chunk[] chunks; // the array of chunks, on which the index is based
60 // number of cell-representations in a chunk
61 // depends on the size of the chunks (may be index chunks or regular data chunks)
62 // each chunk starts with its own ID following the cells data
63 private final int numOfCellRepsInChunk;
65 /**
66 * C-tor for creating CellChunkMap from existing Chunk array, which must be ordered
67 * (decreasingly or increasingly according to parameter "descending")
68 * @param comparator a tool for comparing cells
69 * @param chunks ordered array of index chunk with cell representations
70 * @param min the index of the first cell (usually 0)
71 * @param max number of Cells or the index of the cell after the maximal cell
72 * @param descending the order of the given array
74 public CellChunkMap(Comparator<? super Cell> comparator,
75 Chunk[] chunks, int min, int max, boolean descending) {
76 super(comparator, min, max, descending);
77 this.chunks = chunks;
78 if (chunks != null && chunks.length != 0 && chunks[0] != null) {
79 this.numOfCellRepsInChunk = (chunks[0].size - ChunkCreator.SIZEOF_CHUNK_HEADER) /
80 ClassSize.CELL_CHUNK_MAP_ENTRY;
81 } else { // In case the chunks array was not allocated
82 this.numOfCellRepsInChunk = 0;
86 /* To be used by base (CellFlatMap) class only to create a sub-CellFlatMap
87 * Should be used only to create only CellChunkMap from CellChunkMap */
88 @Override
89 protected CellFlatMap createSubCellFlatMap(int min, int max, boolean descending) {
90 return new CellChunkMap(this.comparator(), this.chunks, min, max, descending);
94 @Override
95 protected Cell getCell(int i) {
96 // get the index of the relevant chunk inside chunk array
97 int chunkIndex = (i / numOfCellRepsInChunk);
98 ByteBuffer block = chunks[chunkIndex].getData();// get the ByteBuffer of the relevant chunk
99 int j = i - chunkIndex * numOfCellRepsInChunk; // get the index of the cell-representation
101 // find inside the offset inside the chunk holding the index, skip bytes for chunk id
102 int offsetInBytes = ChunkCreator.SIZEOF_CHUNK_HEADER + j* ClassSize.CELL_CHUNK_MAP_ENTRY;
104 // find the chunk holding the data of the cell, the chunkID is stored first
105 int chunkId = ByteBufferUtils.toInt(block, offsetInBytes);
106 Chunk chunk = ChunkCreator.getInstance().getChunk(chunkId);
107 if (chunk == null) {
108 // this should not happen
109 throw new IllegalArgumentException("In CellChunkMap, cell must be associated with chunk."
110 + ". We were looking for a cell at index " + i);
113 // find the offset of the data of the cell, skip integer for chunkID, offset is stored second
114 int offsetOfCell = ByteBufferUtils.toInt(block, offsetInBytes + Bytes.SIZEOF_INT);
115 // find the length of the data of the cell, skip two integers for chunkID and offset,
116 // length is stored third
117 int lengthOfCell = ByteBufferUtils.toInt(block, offsetInBytes + 2*Bytes.SIZEOF_INT);
118 // find the seqID of the cell, skip three integers for chunkID, offset, and length
119 // the seqID is plain written as part of the cell representation
120 long cellSeqID = ByteBufferUtils.toLong(block, offsetInBytes + 3*Bytes.SIZEOF_INT);
122 ByteBuffer buf = chunk.getData(); // get the ByteBuffer where the cell data is stored
123 if (buf == null) {
124 // this should not happen
125 throw new IllegalArgumentException("In CellChunkMap, chunk must be associated with ByteBuffer."
126 + " Chunk: " + chunk + " Chunk ID: " + chunk.getId() + ", is from pool: "
127 + chunk.isFromPool() + ". We were looking for a cell at index " + i);
130 return new ByteBufferChunkKeyValue(buf, offsetOfCell, lengthOfCell, cellSeqID);