HBASE-26412 Handle sink failure in RegionReplicationSink (#3815)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / MutableSegment.java
blob6e813433fac69acf2da24104f695d6d6cc5047b1
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.Iterator;
22 import java.util.SortedSet;
23 import java.util.concurrent.atomic.AtomicBoolean;
25 import org.apache.hadoop.hbase.Cell;
26 import org.apache.hadoop.hbase.CellComparator;
27 import org.apache.hadoop.hbase.CellUtil;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.PrivateCellUtil;
30 import org.apache.hadoop.hbase.KeyValue;
31 import org.apache.hadoop.hbase.util.ClassSize;
32 import org.apache.yetus.audience.InterfaceAudience;
34 /**
35 * A mutable segment in memstore, specifically the active segment.
37 @InterfaceAudience.Private
38 public class MutableSegment extends Segment {
40 private final AtomicBoolean flushed = new AtomicBoolean(false);
42 public final static long DEEP_OVERHEAD = ClassSize.align(Segment.DEEP_OVERHEAD
43 + ClassSize.CONCURRENT_SKIPLISTMAP
44 + ClassSize.SYNC_TIMERANGE_TRACKER
45 + ClassSize.REFERENCE
46 + ClassSize.ATOMIC_BOOLEAN);
48 protected MutableSegment(CellSet cellSet, CellComparator comparator,
49 MemStoreLAB memStoreLAB, MemStoreSizing memstoreSizing) {
50 super(cellSet, comparator, memStoreLAB, TimeRangeTracker.create(TimeRangeTracker.Type.SYNC));
51 incMemStoreSize(0, DEEP_OVERHEAD, 0, 0); // update the mutable segment metadata
52 if (memstoreSizing != null) {
53 memstoreSizing.incMemStoreSize(0, DEEP_OVERHEAD, 0, 0);
57 /**
58 * Adds the given cell into the segment
59 * @param cell the cell to add
60 * @param mslabUsed whether using MSLAB
62 public void add(Cell cell, boolean mslabUsed, MemStoreSizing memStoreSizing,
63 boolean sizeAddedPreOperation) {
64 internalAdd(cell, mslabUsed, memStoreSizing, sizeAddedPreOperation);
67 public void upsert(Cell cell, long readpoint, MemStoreSizing memStoreSizing,
68 boolean sizeAddedPreOperation) {
69 internalAdd(cell, false, memStoreSizing, sizeAddedPreOperation);
71 // Get the Cells for the row/family/qualifier regardless of timestamp.
72 // For this case we want to clean up any other puts
73 Cell firstCell = PrivateCellUtil.createFirstOnRowColTS(cell, HConstants.LATEST_TIMESTAMP);
74 SortedSet<Cell> ss = this.tailSet(firstCell);
75 Iterator<Cell> it = ss.iterator();
76 // versions visible to oldest scanner
77 int versionsVisible = 0;
78 while (it.hasNext()) {
79 Cell cur = it.next();
81 if (cell == cur) {
82 // ignore the one just put in
83 continue;
85 // check that this is the row and column we are interested in, otherwise bail
86 if (CellUtil.matchingRows(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
87 // only remove Puts that concurrent scanners cannot possibly see
88 if (cur.getTypeByte() == KeyValue.Type.Put.getCode() && cur.getSequenceId() <= readpoint) {
89 if (versionsVisible >= 1) {
90 // if we get here we have seen at least one version visible to the oldest scanner,
91 // which means we can prove that no scanner will see this version
93 // false means there was a change, so give us the size.
94 // TODO when the removed cell ie.'cur' having its data in MSLAB, we can not release that
95 // area. Only the Cell object as such going way. We need to consider cellLen to be
96 // decreased there as 0 only. Just keeping it as existing code now. We need to know the
97 // removed cell is from MSLAB or not. Will do once HBASE-16438 is in
98 int cellLen = getCellLength(cur);
99 long heapSize = heapSizeChange(cur, true);
100 long offHeapSize = offHeapSizeChange(cur, true);
101 incMemStoreSize(-cellLen, -heapSize, -offHeapSize, -1);
102 if (memStoreSizing != null) {
103 memStoreSizing.decMemStoreSize(cellLen, heapSize, offHeapSize, 1);
105 it.remove();
106 } else {
107 versionsVisible++;
110 } else {
111 // past the row or column, done
112 break;
117 public boolean setInMemoryFlushed() {
118 return flushed.compareAndSet(false, true);
122 * Returns the first cell in the segment
123 * @return the first cell in the segment
125 Cell first() {
126 return this.getCellSet().first();
129 @Override protected long indexEntrySize() {
130 return ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY;