HBASE-26412 Handle sink failure in RegionReplicationSink (#3815)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / CompositeImmutableSegment.java
blob53ef82d5695058b9671eaa900a4a99bb6ee58362
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;
32 /**
33 * The CompositeImmutableSegments is created as a collection of ImmutableSegments and supports
34 * the interface of a single ImmutableSegments.
35 * The CompositeImmutableSegments is planned to be used only as a snapshot,
36 * thus only relevant interfaces are supported
38 @InterfaceAudience.Private
39 public class CompositeImmutableSegment extends ImmutableSegment {
41 private final List<ImmutableSegment> segments;
42 private long keySize = 0;
44 public CompositeImmutableSegment(CellComparator comparator, List<ImmutableSegment> segments) {
45 super(comparator, segments);
46 this.segments = segments;
47 for (ImmutableSegment s : segments) {
48 this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMax());
49 this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMin());
50 this.keySize += s.getDataSize();
54 @Override
55 public List<Segment> getAllSegments() {
56 return new ArrayList<>(segments);
59 @Override
60 public int getNumOfSegments() {
61 return segments.size();
64 /**
65 * @return whether the segment has any cells
67 @Override
68 public boolean isEmpty() {
69 for (ImmutableSegment s : segments) {
70 if (!s.isEmpty()) return false;
72 return true;
75 /**
76 * @return number of cells in segment
78 @Override
79 public int getCellsCount() {
80 int result = 0;
81 for (ImmutableSegment s : segments) {
82 result += s.getCellsCount();
84 return result;
87 /**
88 * Closing a segment before it is being discarded
90 @Override
91 public void close() {
92 for (ImmutableSegment s : segments) {
93 s.close();
97 /**
98 * If the segment has a memory allocator the cell is being cloned to this space, and returned;
99 * otherwise the given cell is returned
100 * @return either the given cell or its clone
102 @Override
103 public Cell maybeCloneWithAllocator(Cell cell, boolean forceCloneOfBigCell) {
104 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
107 @Override
108 public boolean shouldSeek(TimeRange tr, long oldestUnexpiredTS){
109 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
113 * Creates the scanner for the given read point
114 * @return a scanner for the given read point
116 @Override
117 public KeyValueScanner getScanner(long readPoint) {
118 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
122 @Override
123 public List<KeyValueScanner> getScanners(long readPoint) {
124 List<KeyValueScanner> list = new ArrayList<>(segments.size());
125 AbstractMemStore.addToScanners(segments, readPoint, list);
126 return list;
129 @Override
130 public boolean isTagsPresent() {
131 for (ImmutableSegment s : segments) {
132 if (s.isTagsPresent()) return true;
134 return false;
137 @Override
138 public void incScannerCount() {
139 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
142 @Override
143 public void decScannerCount() {
144 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
148 * Setting the CellSet of the segment - used only for flat immutable segment for setting
149 * immutable CellSet after its creation in immutable segment constructor
150 * @return this object
152 @Override
153 protected CompositeImmutableSegment setCellSet(CellSet cellSetOld, CellSet cellSetNew) {
154 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
158 @Override
159 protected long indexEntrySize() {
160 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
163 @Override protected boolean canBeFlattened() {
164 return false;
168 * @return Sum of all cell sizes.
170 @Override
171 public long getDataSize() {
172 return this.keySize;
176 * @return The heap size of this segment.
178 @Override
179 public long getHeapSize() {
180 long result = 0;
181 for (ImmutableSegment s : segments) {
182 result += s.getHeapSize();
184 return result;
188 * Updates the heap size counter of the segment by the given delta
190 @Override
191 public long incMemStoreSize(long delta, long heapOverhead, long offHeapOverhead, int cellsCount) {
192 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
195 @Override
196 public long getMinSequenceId() {
197 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
200 @Override
201 public TimeRangeTracker getTimeRangeTracker() {
202 return this.timeRangeTracker;
205 //*** Methods for SegmentsScanner
206 @Override
207 public Cell last() {
208 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
211 @Override
212 public Iterator<Cell> iterator() {
213 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
216 @Override
217 public SortedSet<Cell> headSet(Cell firstKeyOnRow) {
218 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
221 @Override
222 public int compare(Cell left, Cell right) {
223 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
226 @Override
227 public int compareRows(Cell left, Cell right) {
228 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
232 * @return a set of all cells in the segment
234 @Override
235 protected CellSet getCellSet() {
236 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
239 @Override
240 protected void internalAdd(Cell cell, boolean mslabUsed, MemStoreSizing memstoreSizing,
241 boolean sizeAddedPreOperation) {
242 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
245 @Override
246 protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed,
247 MemStoreSizing memstoreSizing, boolean sizeAddedPreOperation) {
248 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
252 * Returns a subset of the segment cell set, which starts with the given cell
253 * @param firstCell a cell in the segment
254 * @return a subset of the segment cell set, which starts with the given cell
256 @Override
257 protected SortedSet<Cell> tailSet(Cell firstCell) {
258 throw new IllegalStateException("Not supported by CompositeImmutableScanner");
261 // Debug methods
263 * Dumps all cells of the segment into the given log
265 @Override
266 void dump(Logger log) {
267 for (ImmutableSegment s : segments) {
268 s.dump(log);
272 @Override
273 public String toString() {
274 StringBuilder sb =
275 new StringBuilder("This is CompositeImmutableSegment and those are its segments:: ");
276 for (ImmutableSegment s : segments) {
277 sb.append(s.toString());
279 return sb.toString();
282 @Override
283 List<KeyValueScanner> getSnapshotScanners() {
284 List<KeyValueScanner> list = new ArrayList<>(this.segments.size());
285 for (ImmutableSegment segment: this.segments) {
286 list.add(new SnapshotSegmentScanner(segment));
288 return list;