private long getTotalUmcompressedBytes(List<HStoreFile> files) { return files.stream().filter(f -> f != null && f.getReader() != null) .mapToLong(f -> f.getReader().getTotalUncompressedBytes()).sum(); }
/** * <p>It works by processing a compaction that's been written to disk. * * <p>It is usually invoked at the end of a compaction, but might also be * invoked at HStore startup, if the prior execution died midway through. * * <p>Moving the compacted TreeMap into place means: * <pre> * 1) Unload all replaced StoreFile, close and collect list to delete. * 2) Compute new store size * </pre> * * @param compactedFiles list of files that were compacted */ @VisibleForTesting protected void completeCompaction(Collection<HStoreFile> compactedFiles) throws IOException { this.storeSize.set(0L); this.totalUncompressedBytes.set(0L); for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) { StoreFileReader r = hsf.getReader(); if (r == null) { LOG.warn("StoreFile {} has a null Reader", hsf); continue; } this.storeSize.addAndGet(r.length()); this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes()); } }
private void bulkLoadHFile(HStoreFile sf) throws IOException { StoreFileReader r = sf.getReader(); this.storeSize.addAndGet(r.length()); this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes()); // Append the new storefile into the list this.lock.writeLock().lock(); try { this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf)); } finally { // We need the lock, as long as we are updating the storeFiles // or changing the memstore. Let us release it before calling // notifyChangeReadersObservers. See HBASE-4485 for a possible // deadlock scenario that could have happened if continue to hold // the lock. this.lock.writeLock().unlock(); } LOG.info("Loaded HFile " + sf.getFileInfo() + " into store '" + getColumnFamilyName()); if (LOG.isTraceEnabled()) { String traceMessage = "BULK LOAD time,size,store size,store files [" + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]"; LOG.trace(traceMessage); } }
/** * @param path The pathname of the tmp file into which the store was flushed * @param logCacheFlushId * @param status * @return store file created. * @throws IOException */ private HStoreFile commitFile(Path path, long logCacheFlushId, MonitoredTask status) throws IOException { // Write-out finished successfully, move into the right spot Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path); status.setStatus("Flushing " + this + ": reopening flushed file"); HStoreFile sf = createStoreFileAndReader(dstPath); StoreFileReader r = sf.getReader(); this.storeSize.addAndGet(r.length()); this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes()); if (LOG.isInfoEnabled()) { LOG.info("Added " + sf + ", entries=" + r.getEntries() + ", sequenceid=" + logCacheFlushId + ", filesize=" + TraditionalBinaryPrefix.long2String(r.length(), "", 1)); } return sf; }
HStore.this.storeSize.addAndGet(storeFile.getReader().length()); HStore.this.totalUncompressedBytes .addAndGet(storeFile.getReader().getTotalUncompressedBytes()); if (LOG.isInfoEnabled()) { LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
protected HStoreFile createMockStoreFile(final long sizeInBytes, final long seqId) { HStoreFile mockSf = mock(HStoreFile.class); StoreFileReader reader = mock(StoreFileReader.class); String stringPath = "/hbase/testTable/regionA/" + RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random); Path path = new Path(stringPath); when(reader.getSequenceID()).thenReturn(seqId); when(reader.getTotalUncompressedBytes()).thenReturn(sizeInBytes); when(reader.length()).thenReturn(sizeInBytes); when(mockSf.getPath()).thenReturn(path); when(mockSf.excludeFromMinorCompaction()).thenReturn(false); when(mockSf.isReference()).thenReturn(false); // TODO come back to // this when selection takes this into account when(mockSf.getReader()).thenReturn(reader); String toString = MoreObjects.toStringHelper("MockStoreFile") .add("isReference", false) .add("fileSize", StringUtils.humanReadableInt(sizeInBytes)) .add("seqId", seqId) .add("path", stringPath).toString(); when(mockSf.toString()).thenReturn(toString); return mockSf; } }
protected HStoreFile createMockStoreFile(final long sizeInBytes, final long seqId) { HStoreFile mockSf = mock(HStoreFile.class); StoreFileReader reader = mock(StoreFileReader.class); String stringPath = "/hbase/testTable/regionA/" + RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random); Path path = new Path(stringPath); when(reader.getSequenceID()).thenReturn(seqId); when(reader.getTotalUncompressedBytes()).thenReturn(sizeInBytes); when(reader.length()).thenReturn(sizeInBytes); when(mockSf.getPath()).thenReturn(path); when(mockSf.excludeFromMinorCompaction()).thenReturn(false); when(mockSf.isReference()).thenReturn(false); // TODO come back to // this when selection takes this into account when(mockSf.getReader()).thenReturn(reader); String toString = MoreObjects.toStringHelper("MockStoreFile") .add("isReference", false) .add("fileSize", StringUtils.humanReadableInt(sizeInBytes)) .add("seqId", seqId) .add("path", stringPath).toString(); when(mockSf.toString()).thenReturn(toString); return mockSf; } }