if (storeClosing && file.isReferencedInReads()) { LOG.warn("Region closing but StoreFile still has references: file={}, refCount={}", file.getPath(), r.getRefCount()); final long length = r.length(); r.close(true); file.closeStreamReaders(true); + " because of either isCompactedAway=" + file.isCompactedAway() + " or file has reference, isReferencedInReads=" + file.isReferencedInReads() + ", refCount=" + r.getRefCount() + ", skipping for now.");
private static HStoreFile createFile(long size) throws Exception { HStoreFile sf = mock(HStoreFile.class); when(sf.getPath()).thenReturn(new Path("moo")); StoreFileReader r = mock(StoreFileReader.class); when(r.getEntries()).thenReturn(size); when(r.length()).thenReturn(size); when(r.getBloomFilterType()).thenReturn(BloomType.NONE); when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class)); when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(), anyBoolean())).thenReturn(mock(StoreFileScanner.class)); when(sf.getReader()).thenReturn(r); when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty()); return sf; }
@Override public long getFilterEntries() { // Estimate the number of entries as half the original file; this may be wildly inaccurate. return super.getFilterEntries() / 2; } }
protected void setCurrentCell(Cell newVal) throws IOException { this.cur = newVal; if (this.cur != null && this.reader.isBulkLoaded() && !this.reader.isSkipResetSeqId()) { PrivateCellUtil.setSequenceId(cur, this.reader.getSequenceID()); } }
/** * @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; }
Optional<Cell> optionalMidKey = reader.midKey(); if (!optionalMidKey.isPresent()) { return Optional.empty(); Cell firstKey = reader.getFirstKey().get(); Cell lastKey = reader.getLastKey().get();
new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf); reader.loadFileInfo(); reader.loadBloomfilter(); StoreFileScanner scanner = getStoreFileScanner(reader, false, false); reader.close(true); // evict because we are about to delete the file fs.delete(f, true); assertEquals("False negatives: " + falseNeg, 0, falseNeg);
metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo()); this.reader.setSkipResetSeqId(skipResetSeqId); this.reader.setBulkLoaded(true); this.reader.setSequenceID(this.sequenceid); this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b)); BloomType hfileBloomType = reader.getBloomFilterType(); if (cfBloomType != BloomType.NONE) { reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META); if (hfileBloomType != cfBloomType) { LOG.info("HFile Bloom filter type for " + reader.getHFileReader().getName() + ": " + hfileBloomType + ", but " + cfBloomType + " specified in column family " + "configuration"); + reader.getHFileReader().getName()); reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META); firstKey = reader.getFirstKey(); lastKey = reader.getLastKey(); comparator = reader.getComparator();
new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf); reader.loadFileInfo(); reader.loadBloomfilter(); String row = String.format(localFormatter, i); byte[] rowKey = Bytes.toBytes(row); boolean exists = reader.passesDeleteFamilyBloomFilter(rowKey, 0, rowKey.length); if (i % 2 == 0) { if (!exists) assertEquals(1000, reader.getDeleteFamilyCnt()); reader.close(true); // evict because we are about to delete the file fs.delete(f, true); assertEquals("False negatives: " + falseNeg, 0, falseNeg);
long keyCount = r.getEntries(); fd.maxKeyCount += keyCount; Map<byte[], byte[]> fileInfo = r.loadFileInfo(); byte[] tmp = null; if (r.isBulkLoaded()) { fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, r.getSequenceID()); (file.getPath() == null? null: file.getPath().getName()), keyCount, r.getBloomFilterType().toString(), TraditionalBinaryPrefix.long2String(r.length(), "", 1), r.getHFileReader().getDataBlockEncoding(), compactionCompression, seqNum,
hsf.initReader(); StoreFileReader reader = hsf.getReader(); reader.loadFileInfo(); KeyValueScanner scanner = reader.getStoreFileScanner(true, true, false, hsf.getMaxMemStoreTS(), 0, false); USE_TAG = reader.getHFileReader().getFileContext().isIncludesTags(); int majorVersion = reader.getHFileVersion(); comp.useHBaseChecksum = majorVersion > 2 || (majorVersion == 2 && reader.getHFileMinorVersion() >= HFileReaderImpl.MINOR_VERSION_WITH_CHECKSUM); comp.checkStatistics(scanner, kvLimit); if (doVerify) { reader.close(cacheConf.shouldEvictOnClose());
StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf); reader.loadFileInfo(); reader.loadBloomfilter(); assertEquals(bt[x], reader.getBloomFilterType()); if (bt[x] == BloomType.ROWPREFIX_FIXED_LENGTH) { assertEquals(prefixLength, reader.getPrefixLength()); assertEquals("null", Bytes.toStringBinary(reader.getDelimiter())); } else if (bt[x] == BloomType.ROWPREFIX_DELIMITED){ assertEquals(-1, reader.getPrefixLength()); assertEquals(delimiter, Bytes.toStringBinary(reader.getDelimiter())); assertEquals(expKeys, reader.getGeneralBloomFilter().getKeyCount()); StoreFileScanner scanner = getStoreFileScanner(reader); HStore store = mock(HStore.class); reader.close(true); // evict because we are about to delete the file fs.delete(f, true); assertEquals("False negatives: " + falseNeg, 0, falseNeg);
for (HStoreFile storefile : storeFiles) { StoreFileReader reader = storefile.getReader(); reader.loadFileInfo(); reader.loadBloomfilter(); assertEquals(num_unique_rows * duplicate_multiplier, reader.getEntries()); assertEquals(num_unique_rows, reader.getFilterEntries()); for (HStoreFile storefile : storeFiles) { StoreFileReader reader = storefile.getReader(); reader.loadFileInfo(); reader.loadBloomfilter(); assertEquals(num_unique_rows * duplicate_multiplier * num_storefiles, reader.getEntries()); assertEquals(num_unique_rows, reader.getFilterEntries());
HStoreFile storeFile = createStoreFileAndReader(storeFileInfo); storeFiles.add(storeFile); HStore.this.storeSize.addAndGet(storeFile.getReader().length()); HStore.this.totalUncompressedBytes .addAndGet(storeFile.getReader().getTotalUncompressedBytes()); if (LOG.isInfoEnabled()) { LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() + " added " + storeFile + ", entries=" + storeFile.getReader().getEntries() + ", sequenceid=" + +storeFile.getReader().getSequenceID() + ", filesize=" + TraditionalBinaryPrefix.long2String(storeFile.getReader().length(), "", 1));
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; } }
byte [] midRow = CellUtil.cloneRow(reader.midKey().get()); byte [] finalRow = CellUtil.cloneRow(reader.getLastKey().get()); hsf.closeStoreFile(true); HFileScanner s = refHsf.getReader().getScanner(false, false); Cell kv = null; for (boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
private void checkHalfHFile(final HRegionFileSystem regionFs, final HStoreFile f) throws IOException { f.initReader(); Cell midkey = f.getReader().midKey().get(); KeyValue midKV = (KeyValue)midkey; byte [] midRow = CellUtil.cloneRow(midKV); HFileScanner topScanner = top.getScanner(false, false); while ((!topScanner.isSeeked() && topScanner.seekTo()) || (topScanner.isSeeked() && topScanner.next())) { HFileScanner bottomScanner = bottom.getScanner(false, false); while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) || bottomScanner.next()) { topScanner = top.getScanner(false, false); KeyValue.KeyOnlyKeyValue keyOnlyKV = new KeyValue.KeyOnlyKeyValue(); while ((!topScanner.isSeeked() && topScanner.seekTo()) || bottom = bottomF.getReader(); first = true; bottomScanner = bottom.getScanner(false, false); while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) || bottomScanner.next()) { top.close(true); // evict since we are about to delete the file bottom.close(true); // evict since we are about to delete the file
/** * @deprecated Do not write further code which depends on this call. Instead * use getStoreFileScanner() which uses the StoreFileScanner class/interface * which is the preferred way to scan a store with higher level concepts. * * @param cacheBlocks should we cache the blocks? * @param pread use pread (for concurrent small readers) * @return the underlying HFileScanner */ @Deprecated public HFileScanner getScanner(boolean cacheBlocks, boolean pread) { return getScanner(cacheBlocks, pread, false); }
@Override public Path getFilePath() { return reader.getHFileReader().getPath(); }
if (useBloom) { if (reader.getBloomFilterType() == BloomType.ROWCOL) { haveToSeek = reader.passesGeneralRowColBloomFilter(kv); } else if (canOptimizeForNonNullColumn && ((PrivateCellUtil.isDeleteFamily(kv) haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength()); long maxTimestampInFile = reader.getMaxTimestamp(); long seekTimestamp = kv.getTimestamp(); if (seekTimestamp > maxTimestampInFile) {