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.util.Iterator;
022import java.util.SortedSet;
023import java.util.concurrent.atomic.AtomicBoolean;
024
025import org.apache.hadoop.hbase.Cell;
026import org.apache.hadoop.hbase.CellComparator;
027import org.apache.hadoop.hbase.CellUtil;
028import org.apache.hadoop.hbase.HConstants;
029import org.apache.hadoop.hbase.PrivateCellUtil;
030import org.apache.hadoop.hbase.KeyValue;
031import org.apache.yetus.audience.InterfaceAudience;
032import org.apache.hadoop.hbase.util.ClassSize;
033
034import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
035
036/**
037 * A mutable segment in memstore, specifically the active segment.
038 */
039@InterfaceAudience.Private
040public class MutableSegment extends Segment {
041
042  private final AtomicBoolean flushed = new AtomicBoolean(false);
043
044  public final static long DEEP_OVERHEAD = ClassSize.align(Segment.DEEP_OVERHEAD
045      + ClassSize.CONCURRENT_SKIPLISTMAP
046      + ClassSize.SYNC_TIMERANGE_TRACKER
047      + ClassSize.REFERENCE
048      + ClassSize.ATOMIC_BOOLEAN);
049
050  protected MutableSegment(CellSet cellSet, CellComparator comparator,
051      MemStoreLAB memStoreLAB, MemStoreSizing memstoreSizing) {
052    super(cellSet, comparator, memStoreLAB, TimeRangeTracker.create(TimeRangeTracker.Type.SYNC));
053    incMemStoreSize(0, DEEP_OVERHEAD, 0, 0); // update the mutable segment metadata
054    if (memstoreSizing != null) {
055      memstoreSizing.incMemStoreSize(0, DEEP_OVERHEAD, 0, 0);
056    }
057  }
058
059  /**
060   * Adds the given cell into the segment
061   * @param cell the cell to add
062   * @param mslabUsed whether using MSLAB
063   */
064  public void add(Cell cell, boolean mslabUsed, MemStoreSizing memStoreSizing,
065      boolean sizeAddedPreOperation) {
066    internalAdd(cell, mslabUsed, memStoreSizing, sizeAddedPreOperation);
067  }
068
069  public void upsert(Cell cell, long readpoint, MemStoreSizing memStoreSizing,
070      boolean sizeAddedPreOperation) {
071    internalAdd(cell, false, memStoreSizing, sizeAddedPreOperation);
072
073    // Get the Cells for the row/family/qualifier regardless of timestamp.
074    // For this case we want to clean up any other puts
075    Cell firstCell = PrivateCellUtil.createFirstOnRowColTS(cell, HConstants.LATEST_TIMESTAMP);
076    SortedSet<Cell> ss = this.tailSet(firstCell);
077    Iterator<Cell> it = ss.iterator();
078    // versions visible to oldest scanner
079    int versionsVisible = 0;
080    while (it.hasNext()) {
081      Cell cur = it.next();
082
083      if (cell == cur) {
084        // ignore the one just put in
085        continue;
086      }
087      // check that this is the row and column we are interested in, otherwise bail
088      if (CellUtil.matchingRows(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
089        // only remove Puts that concurrent scanners cannot possibly see
090        if (cur.getTypeByte() == KeyValue.Type.Put.getCode() && cur.getSequenceId() <= readpoint) {
091          if (versionsVisible >= 1) {
092            // if we get here we have seen at least one version visible to the oldest scanner,
093            // which means we can prove that no scanner will see this version
094
095            // false means there was a change, so give us the size.
096            // TODO when the removed cell ie.'cur' having its data in MSLAB, we can not release that
097            // area. Only the Cell object as such going way. We need to consider cellLen to be
098            // decreased there as 0 only. Just keeping it as existing code now. We need to know the
099            // removed cell is from MSLAB or not. Will do once HBASE-16438 is in
100            int cellLen = getCellLength(cur);
101            long heapSize = heapSizeChange(cur, true);
102            long offHeapSize = offHeapSizeChange(cur, true);
103            incMemStoreSize(-cellLen, -heapSize, -offHeapSize, -1);
104            if (memStoreSizing != null) {
105              memStoreSizing.decMemStoreSize(cellLen, heapSize, offHeapSize, 1);
106            }
107            it.remove();
108          } else {
109            versionsVisible++;
110          }
111        }
112      } else {
113        // past the row or column, done
114        break;
115      }
116    }
117  }
118
119  public boolean setInMemoryFlushed() {
120    return flushed.compareAndSet(false, true);
121  }
122
123  /**
124   * Returns the first cell in the segment
125   * @return the first cell in the segment
126   */
127  @VisibleForTesting
128  Cell first() {
129    return this.getCellSet().first();
130  }
131
132  @Override protected long indexEntrySize() {
133      return ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY;
134  }
135}