@Override public long getHeapSize() { return this.mss.getHeapSize(); }
@Override public long heapSize() { MemStoreSize memstoreSize = this.memstore.size(); return DEEP_OVERHEAD + memstoreSize.getHeapSize(); }
private boolean isFlushSize(MemStoreSize size) { return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize(); }
/** * Package private constructor. */ MemStoreSize(MemStoreSize memStoreSize) { this.dataSize = memStoreSize.getDataSize(); this.heapSize = memStoreSize.getHeapSize(); this.offHeapSize = memStoreSize.getOffHeapSize(); this.cellsCount = memStoreSize.getCellsCount(); }
ThreadSafeMemStoreSizing(MemStoreSize mss) { this(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), mss.getCellsCount()); }
NonThreadSafeMemStoreSizing(MemStoreSize mss) { this(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), mss.getCellsCount()); }
/** * 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)); }
void incGlobalMemStoreSize(MemStoreSize mss) { incGlobalMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize()); }
/** * Increase the size of mem store in this region and the size of global mem * store */ void incMemStoreSize(MemStoreSize mss) { incMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), mss.getCellsCount()); }
default long decMemStoreSize(MemStoreSize delta) { return incMemStoreSize(-delta.getDataSize(), -delta.getHeapSize(), -delta.getOffHeapSize(), -delta.getCellsCount()); }
void decrMemStoreSize(MemStoreSize mss) { decrMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), mss.getCellsCount()); }
default long incMemStoreSize(MemStoreSize delta) { return incMemStoreSize(delta.getDataSize(), delta.getHeapSize(), delta.getOffHeapSize(), delta.getCellsCount()); }
protected boolean shouldFlush(HStore store) { if (store.getMemStoreSize().getHeapSize() + store.getMemStoreSize().getOffHeapSize() > this.flushSizeLowerBound) { LOG.debug("Flush {} of {}; " + "heap memstoreSize={} +" + "off heap memstoreSize={} > memstore lowerBound={}" , store.getColumnFamilyName() , region.getRegionInfo().getEncodedName() , store.getMemStoreSize().getHeapSize() , store.getMemStoreSize().getOffHeapSize() , this.flushSizeLowerBound ); return true; } return false; } }
protected Segment(CellComparator comparator, List<ImmutableSegment> segments, TimeRangeTracker trt) { long dataSize = 0; long heapSize = 0; long OffHeapSize = 0; int cellsCount = 0; for (Segment segment : segments) { MemStoreSize memStoreSize = segment.getMemStoreSize(); dataSize += memStoreSize.getDataSize(); heapSize += memStoreSize.getHeapSize(); OffHeapSize += memStoreSize.getOffHeapSize(); cellsCount += memStoreSize.getCellsCount(); } this.comparator = comparator; this.updatesLock = new ReentrantReadWriteLock(); // Do we need to be thread safe always? What if ImmutableSegment? // DITTO for the TimeRangeTracker below. this.memStoreSizing = new ThreadSafeMemStoreSizing(dataSize, heapSize, OffHeapSize, cellsCount); this.timeRangeTracker = trt; }
memStoreSizing.getMemStoreSize().getHeapSize()); for (int i = 0; i < count; i++) { memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memStoreSizing); memStoreSizing.getMemStoreSize().getHeapSize()); memStoreSizing.getMemStoreSize().getHeapSize()); final int seconds = 30; LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
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())); } }
mss = store.memstore.getFlushableSize(); assertEquals(0, mss.getDataSize()); assertEquals(MutableSegment.DEEP_OVERHEAD, mss.getHeapSize()); return null;
@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()); } }