001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.util.List; 021import java.util.concurrent.atomic.AtomicInteger; 022 023import org.apache.hadoop.hbase.Cell; 024import org.apache.yetus.audience.InterfaceAudience; 025 026/** 027 * A MemStoreLAB implementation which wraps N MemStoreLABs. Its main duty is in proper managing the 028 * close of the individual MemStoreLAB. This is treated as an immutable one and so do not allow to 029 * add any more Cells into it. {@link #copyCellInto(Cell)} throws Exception 030 */ 031@InterfaceAudience.Private 032public class ImmutableMemStoreLAB implements MemStoreLAB { 033 034 private final AtomicInteger openScannerCount = new AtomicInteger(); 035 private volatile boolean closed = false; 036 037 private final List<MemStoreLAB> mslabs; 038 039 public ImmutableMemStoreLAB(List<MemStoreLAB> mslabs) { 040 this.mslabs = mslabs; 041 } 042 043 @Override 044 public Cell copyCellInto(Cell cell) { 045 throw new IllegalStateException("This is an Immutable MemStoreLAB."); 046 } 047 048 /** 049 * The process of merging assumes all cells are allocated on mslab. 050 * There is a rare case in which the first immutable segment, 051 * participating in a merge, is a CSLM. 052 * Since the CSLM hasn't been flattened yet, and there is no point in flattening it (since it is 053 * going to be merged), its big cells (for whom size > maxAlloc) must be copied into mslab. 054 * This method copies the passed cell into the first mslab in the mslabs list, 055 * returning either a new cell instance over the copied data, 056 * or null when this cell cannt be copied. 057 */ 058 @Override 059 public Cell forceCopyOfBigCellInto(Cell cell) { 060 MemStoreLAB mslab = this.mslabs.get(0); 061 return mslab.forceCopyOfBigCellInto(cell); 062 } 063 064 /* Returning a new pool chunk, without replacing current chunk, 065 ** meaning MSLABImpl does not make the returned chunk as CurChunk. 066 ** The space on this chunk will be allocated externally. 067 ** The interface is only for external callers. 068 */ 069 @Override 070 public Chunk getNewExternalChunk(ChunkCreator.ChunkType chunkType) { 071 MemStoreLAB mslab = this.mslabs.get(0); 072 return mslab.getNewExternalChunk(chunkType); 073 } 074 075 /* Returning a new chunk, without replacing current chunk, 076 ** meaning MSLABImpl does not make the returned chunk as CurChunk. 077 ** The space on this chunk will be allocated externally. 078 ** The interface is only for external callers. 079 */ 080 @Override 081 public Chunk getNewExternalChunk(int size) { 082 MemStoreLAB mslab = this.mslabs.get(0); 083 return mslab.getNewExternalChunk(size); 084 } 085 086 @Override 087 public void close() { 088 // 'openScannerCount' here tracks the scanners opened on segments which directly refer to this 089 // MSLAB. The individual MSLABs this refers also having its own 'openScannerCount'. The usage of 090 // the variable in close() and decScannerCount() is as as that in HeapMemstoreLAB. Here the 091 // close just delegates the call to the individual MSLABs. The actual return of the chunks to 092 // MSLABPool will happen within individual MSLABs only (which is at the leaf level). 093 // Say an ImmutableMemStoreLAB is created over 2 HeapMemStoreLABs at some point and at that time 094 // both of them were referred by ongoing scanners. So they have > 0 'openScannerCount'. Now over 095 // the new Segment some scanners come in and this MSLABs 'openScannerCount' also goes up and 096 // then come down on finish of scanners. Now a close() call comes to this Immutable MSLAB. As 097 // it's 'openScannerCount' is zero it will call close() on both of the Heap MSLABs. Say by that 098 // time the old scanners on one of the MSLAB got over where as on the other, still an old 099 // scanner is going on. The call close() on that MSLAB will not close it immediately but will 100 // just mark it for closure as it's 'openScannerCount' still > 0. Later once the old scan is 101 // over, the decScannerCount() call will do the actual close and return of the chunks. 102 this.closed = true; 103 // When there are still on going scanners over this MSLAB, we will defer the close until all 104 // scanners finish. We will just mark it for closure. See #decScannerCount(). This will be 105 // called at end of every scan. When it is marked for closure and scanner count reached 0, we 106 // will do the actual close then. 107 checkAndCloseMSLABs(openScannerCount.get()); 108 } 109 110 private void checkAndCloseMSLABs(int openScanners) { 111 if (openScanners == 0) { 112 for (MemStoreLAB mslab : this.mslabs) { 113 mslab.close(); 114 } 115 } 116 } 117 118 @Override 119 public void incScannerCount() { 120 this.openScannerCount.incrementAndGet(); 121 } 122 123 @Override 124 public void decScannerCount() { 125 int count = this.openScannerCount.decrementAndGet(); 126 if (this.closed) { 127 checkAndCloseMSLABs(count); 128 } 129 } 130 131 @Override 132 public boolean isOnHeap() { 133 return !isOffHeap(); 134 } 135 136 @Override 137 public boolean isOffHeap() { 138 return ChunkCreator.getInstance().isOffheap(); 139 } 140 141 142}