HBASE-23949 refactor loadBalancer implements for rsgroup balance by table to achieve...
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / CompositeImmutableSegment.java
blob19647faa9de1b6b9e58b4be960be6ff7b8e519d8
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, Version 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 KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
19 package org.apache.hadoop.hbase.regionserver;
21 import java.util.ArrayList;
22 import java.util.Iterator;
23 import java.util.List;
24 import java.util.SortedSet;
26 import org.apache.hadoop.hbase.Cell;
27 import org.apache.hadoop.hbase.CellComparator;
28 import org.apache.hadoop.hbase.io.TimeRange;
29 import org.apache.yetus.audience.InterfaceAudience;
30 import org.slf4j.Logger;
31 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
33 /**
34 * The CompositeImmutableSegments is created as a collection of ImmutableSegments and supports
35 * the interface of a single ImmutableSegments.
36 * The CompositeImmutableSegments is planned to be used only as a snapshot,
37 * thus only relevant interfaces are supported
39 @InterfaceAudience.Private
40 public class CompositeImmutableSegment extends ImmutableSegment {
42 private final List<ImmutableSegment> segments;
43 private long keySize = 0;
45 public CompositeImmutableSegment(CellComparator comparator, List<ImmutableSegment> segments) {
46 super(comparator, segments);
47 this.segments = segments;
48 for (ImmutableSegment s : segments) {
49 this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMax());
50 this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMin());
51 this.keySize += s.getDataSize();
55 @VisibleForTesting
56 @Override
57 public List<Segment> getAllSegments() {
58 return new ArrayList<>(segments);
61 @Override
62 public int getNumOfSegments() {
63 return segments.size();
66 /**
67 * @return whether the segment has any cells
69 @Override
70 public boolean isEmpty() {
71 for (ImmutableSegment s : segments) {
72 if (!s.isEmpty()) return false;
74 return true;
77 /**
78 * @return number of cells in segment
80 @Override
81 public int getCellsCount() {
82 int result = 0;
83 for (ImmutableSegment s : segments) {
84 result += s.getCellsCount();
86 return result;
89 /**
90 * Closing a segment before it is being discarded
92 @Override
93 public void close() {
94 for (ImmutableSegment s : segments) {
95 s.close();
99 /**
100 * If the segment has a memory allocator the cell is being cloned to this space, and returned;
101 * otherwise the given cell is returned
102 * @return either the given cell or its clone
104 @Override
105 public Cell maybeCloneWithAllocator(Cell cell, boolean forceCloneOfBigCell) {
106 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
109 @Override
110 public boolean shouldSeek(TimeRange tr, long oldestUnexpiredTS){
111 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
115 * Creates the scanner for the given read point
116 * @return a scanner for the given read point
118 @Override
119 public KeyValueScanner getScanner(long readPoint) {
120 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
124 @Override
125 public List<KeyValueScanner> getScanners(long readPoint) {
126 List<KeyValueScanner> list = new ArrayList<>(segments.size());
127 AbstractMemStore.addToScanners(segments, readPoint, list);
128 return list;
131 @Override
132 public boolean isTagsPresent() {
133 for (ImmutableSegment s : segments) {
134 if (s.isTagsPresent()) return true;
136 return false;
139 @Override
140 public void incScannerCount() {
141 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
144 @Override
145 public void decScannerCount() {
146 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
150 * Setting the CellSet of the segment - used only for flat immutable segment for setting
151 * immutable CellSet after its creation in immutable segment constructor
152 * @return this object
154 @Override
155 protected CompositeImmutableSegment setCellSet(CellSet cellSetOld, CellSet cellSetNew) {
156 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
160 @Override
161 protected long indexEntrySize() {
162 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
165 @Override protected boolean canBeFlattened() {
166 return false;
170 * @return Sum of all cell sizes.
172 @Override
173 public long getDataSize() {
174 return this.keySize;
178 * @return The heap size of this segment.
180 @Override
181 public long getHeapSize() {
182 long result = 0;
183 for (ImmutableSegment s : segments) {
184 result += s.getHeapSize();
186 return result;
190 * Updates the heap size counter of the segment by the given delta
192 @Override
193 public long incMemStoreSize(long delta, long heapOverhead, long offHeapOverhead, int cellsCount) {
194 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
197 @Override
198 public long getMinSequenceId() {
199 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
202 @Override
203 public TimeRangeTracker getTimeRangeTracker() {
204 return this.timeRangeTracker;
207 //*** Methods for SegmentsScanner
208 @Override
209 public Cell last() {
210 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
213 @Override
214 public Iterator<Cell> iterator() {
215 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
218 @Override
219 public SortedSet<Cell> headSet(Cell firstKeyOnRow) {
220 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
223 @Override
224 public int compare(Cell left, Cell right) {
225 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
228 @Override
229 public int compareRows(Cell left, Cell right) {
230 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
234 * @return a set of all cells in the segment
236 @Override
237 protected CellSet getCellSet() {
238 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
241 @Override
242 protected void internalAdd(Cell cell, boolean mslabUsed, MemStoreSizing memstoreSizing,
243 boolean sizeAddedPreOperation) {
244 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
247 @Override
248 protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed,
249 MemStoreSizing memstoreSizing, boolean sizeAddedPreOperation) {
250 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
254 * Returns a subset of the segment cell set, which starts with the given cell
255 * @param firstCell a cell in the segment
256 * @return a subset of the segment cell set, which starts with the given cell
258 @Override
259 protected SortedSet<Cell> tailSet(Cell firstCell) {
260 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
263 // Debug methods
265 * Dumps all cells of the segment into the given log
267 @Override
268 void dump(Logger log) {
269 for (ImmutableSegment s : segments) {
270 s.dump(log);
274 @Override
275 public String toString() {
276 StringBuilder sb =
277 new StringBuilder("This is CompositeImmutableSegment and those are its segments:: ");
278 for (ImmutableSegment s : segments) {
279 sb.append(s.toString());
281 return sb.toString();
284 @Override
285 List<KeyValueScanner> getSnapshotScanners() {
286 List<KeyValueScanner> list = new ArrayList<>(this.segments.size());
287 for (ImmutableSegment segment: this.segments) {
288 list.add(new SnapshotSegmentScanner(segment));
290 return list;