@Override public MemStoreSize getMemStoreSize() { return this.memStoreSizing.getMemStoreSize(); }
private void requestFlushIfNeeded() throws RegionTooBusyException { if(isFlushSize(this.memStoreSizing.getMemStoreSize())) { requestFlush(); } }
public MemStoreSize getPipelineSize() { MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing(); LinkedList<? extends Segment> localCopy = readOnlyCopy; for (Segment segment : localCopy) { memStoreSizing.incMemStoreSize(segment.getMemStoreSize()); } return memStoreSizing.getMemStoreSize(); }
/** * Be careful, this method will drop all data in the memstore of this region. * Currently, this method is used to drop memstore to prevent memory leak * when replaying recovered.edits while opening region. */ public MemStoreSize dropMemStoreContents() throws IOException { MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing(); this.updatesLock.writeLock().lock(); try { for (HStore s : stores.values()) { MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM); LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region " + this.getRegionInfo().getRegionNameAsString() + " , dropped memstoresize: [" + memStoreSize + " }"); totalFreedSize.incMemStoreSize(memStoreSize); } return totalFreedSize.getMemStoreSize(); } finally { this.updatesLock.writeLock().unlock(); } }
/** * Utility method broken out of internalPrepareFlushCache so that method is smaller. */ private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) { if (!LOG.isInfoEnabled()) { return; } // Log a fat line detailing what is being flushed. StringBuilder perCfExtras = null; if (!isAllFamilies(storesToFlush)) { perCfExtras = new StringBuilder(); for (HStore store: storesToFlush) { MemStoreSize mss = store.getFlushableSize(); perCfExtras.append("; ").append(store.getColumnFamilyName()); perCfExtras.append("={dataSize=") .append(StringUtils.byteDesc(mss.getDataSize())); perCfExtras.append(", heapSize=") .append(StringUtils.byteDesc(mss.getHeapSize())); perCfExtras.append(", offHeapSize=") .append(StringUtils.byteDesc(mss.getOffHeapSize())); perCfExtras.append("}"); } } MemStoreSize mss = this.memStoreSizing.getMemStoreSize(); LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() + " column families," + " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) + " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) + ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") + ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId)); }
LOG.info("memstore1 estimated size={}", memStoreSizing.getMemStoreSize().getDataSize() + memStoreSizing.getMemStoreSize().getHeapSize()); for (int i = 0; i < count; i++) { memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memStoreSizing); memStoreSizing.getMemStoreSize().getDataSize() + memStoreSizing.getMemStoreSize().getHeapSize()); memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, new byte[i]), memStoreSizing); LOG.info("memstore2 estimated size={}", memStoreSizing.getMemStoreSize().getDataSize() + memStoreSizing.getMemStoreSize().getHeapSize()); final int seconds = 30; LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
this.updatesLock.writeLock().unlock(); return totalFreedSize.getMemStoreSize();
/** * @return Total memory occupied by this MemStore. This won't include any size occupied by the * snapshot. We assume the snapshot will get cleared soon. This is not thread safe and * the memstore may be changed while computing its size. It is the responsibility of the * caller to make sure this doesn't happen. */ @Override public MemStoreSize size() { MemStoreSizing memstoreSizing = new NonThreadSafeMemStoreSizing(); memstoreSizing.incMemStoreSize(getActive().getMemStoreSize()); for (Segment item : pipeline.getSegments()) { memstoreSizing.incMemStoreSize(item.getMemStoreSize()); } return memstoreSizing.getMemStoreSize(); }
MemStoreSize mss = newMemstoreAccounting.getMemStoreSize(); region.addMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), mss.getCellsCount());
+ getRegionInfo().getRegionNameAsString() + " " + store + ". flushableSize expected=0, actual={" + mss + "}. Current memStoreSize=" + this.memStoreSizing.getMemStoreSize() + ". Maybe a coprocessor " + "operation failed and left the memstore in a partially updated state.", null); decrMemStoreSize(this.memStoreSizing.getMemStoreSize()); } else if (this.memStoreSizing.getDataSize() != 0) { LOG.error("Memstore data size is {}", this.memStoreSizing.getDataSize());
private long addRowsByKeysDataSize(final AbstractMemStore hmc, String[] keys) { byte[] fam = Bytes.toBytes("testfamily"); byte[] qf = Bytes.toBytes("testqualifier"); MemStoreSizing memstoreSizing = new NonThreadSafeMemStoreSizing(); for (int i = 0; i < keys.length; i++) { long timestamp = System.currentTimeMillis(); Threads.sleep(1); // to make sure each kv gets a different ts byte[] row = Bytes.toBytes(keys[i]); byte[] val = Bytes.toBytes(keys[i] + i); KeyValue kv = new KeyValue(row, fam, qf, timestamp, val); hmc.add(kv, memstoreSizing); LOG.debug("added kv: " + kv.getKeyString() + ", timestamp" + kv.getTimestamp()); } MemStoreSize mss = memstoreSizing.getMemStoreSize(); regionServicesForStores.addMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), mss.getCellsCount()); return mss.getDataSize(); }
/** * @return statistics about the current load of the region */ public ClientProtos.RegionLoadStats getLoadStatistics() { if (!regionStatsEnabled) { return null; } ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder(); stats.setMemStoreLoad((int) (Math.min(100, (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize))); if (rsServices.getHeapMemoryManager() != null) { // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM, // so we could just do the calculation below and we'll get a 0. // treating it as a special case analogous to no HMM instead so that it can be // programatically treated different from using <1% of heap. final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent(); if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) { stats.setHeapOccupancy((int)(occupancy * 100)); } } stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100 : rsServices.getCompactionPressure() * 100)); return stats.build(); }
void checkResources() throws RegionTooBusyException { // If catalog region, do not impose resource constraints or block updates. if (this.getRegionInfo().isMetaRegion()) return; MemStoreSize mss = this.memStoreSizing.getMemStoreSize(); if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) { blockedRequestsCount.increment(); requestFlush(); // Don't print current limit because it will vary too much. The message is used as a key // over in RetriesExhaustedWithDetailsException processing. throw new RegionTooBusyException("Over memstore limit=" + org.apache.hadoop.hbase.procedure2.util.StringUtils.humanSize(this.blockingMemStoreSize) + ", regionName=" + (this.getRegionInfo() == null? "unknown": this.getRegionInfo().getEncodedName()) + ", server=" + (this.getRegionServerServices() == null? "unknown": this.getRegionServerServices().getServerName())); } }
MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize(); this.decrMemStoreSize(mss); MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize(); long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize(); String msg = "Finished flush of" + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/" + mss.getDataSize()
this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize()); this.prepareFlushResult = null; writestate.flushing = false; this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
@Override public MemStoreSize getFlushableSize() { MemStoreSize mss = getSnapshotSize(); if (mss.getDataSize() == 0) { // if snapshot is empty the tail of the pipeline (or everything in the memstore) is flushed if (compositeSnapshot) { MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing(pipeline.getPipelineSize()); MutableSegment currActive = getActive(); if(!currActive.isEmpty()) { memStoreSizing.incMemStoreSize(currActive.getMemStoreSize()); } mss = memStoreSizing.getMemStoreSize(); } else { mss = pipeline.getTailSize(); } } return mss.getDataSize() > 0? mss: getActive().getMemStoreSize(); }
closeRegionOperation(); if (!mutations.isEmpty()) { this.incMemStoreSize(memstoreAccounting.getMemStoreSize()); requestFlushIfNeeded();
incMemStoreSize(memstoreAccounting.getMemStoreSize()); requestFlushIfNeeded(); closeRegionOperation(op);
assertEquals(kvSize.getMemStoreSize(), mss); CSLMImmutableSegment.DEEP_OVERHEAD_CSLM - MutableSegment.DEEP_OVERHEAD, 0, 0); mss = store.memstore.getFlushableSize(); assertEquals(kvSize.getMemStoreSize(), mss); MemStoreSizing kvSize2 = new NonThreadSafeMemStoreSizing(); store.add(new KeyValue(row, family, qf2, 2, (byte[]) null), kvSize2); assertEquals(kvSize.getMemStoreSize(), mss); ffs.fault.set(false); flushStore(store, id++); mss = store.memstore.getFlushableSize(); assertEquals(kvSize2.getMemStoreSize(), mss); flushStore(store, id++); mss = store.memstore.getFlushableSize();
@Test public void testPutSameCell() { byte[] bytes = Bytes.toBytes(getName()); KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes); MemStoreSizing sizeChangeForFirstCell = new NonThreadSafeMemStoreSizing(); this.memstore.add(kv, sizeChangeForFirstCell); MemStoreSizing sizeChangeForSecondCell = new NonThreadSafeMemStoreSizing(); this.memstore.add(kv, sizeChangeForSecondCell); // make sure memstore size increase won't double-count MSLAB chunk size assertEquals(Segment.getCellLength(kv), sizeChangeForFirstCell.getMemStoreSize().getDataSize()); Segment segment = this.memstore.getActive(); MemStoreLAB msLab = segment.getMemStoreLAB(); if (msLab != null) { // make sure memstore size increased even when writing the same cell, if using MSLAB assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getMemStoreSize().getDataSize()); // make sure chunk size increased even when writing the same cell, if using MSLAB if (msLab instanceof MemStoreLABImpl) { // since we add the chunkID at the 0th offset of the chunk and the // chunkid is an int we need to account for those 4 bytes assertEquals(2 * Segment.getCellLength(kv) + Bytes.SIZEOF_INT, ((MemStoreLABImpl) msLab).getCurrentChunk().getNextFreeOffset()); } } else { // make sure no memstore size change w/o MSLAB assertEquals(0, sizeChangeForSecondCell.getMemStoreSize().getDataSize()); assertEquals(0, sizeChangeForSecondCell.getMemStoreSize().getHeapSize()); } }