diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index d015ceb2d457..dd0cd5bc30cc 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -1088,10 +1088,11 @@ possible configurations would overwhelm and obscure the important. hfile.format.version - 3 + 4 The HFile format version to use for new files. - Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags). - Also see the configuration 'hbase.replication.rpc.codec'. + Version 4 is the default for HBase 4.x and introduces the multi-tenant HFile layout while + remaining backward compatible with older readers. Also see the configuration + 'hbase.replication.rpc.codec'. diff --git a/hbase-protocol-shaded/src/main/protobuf/server/io/HFile.proto b/hbase-protocol-shaded/src/main/protobuf/server/io/HFile.proto index fd1b9b3680d8..eea1206527ce 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/io/HFile.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/io/HFile.proto @@ -51,4 +51,7 @@ message FileTrailerProto { optional string comparator_class_name = 11; optional uint32 compression_codec = 12; optional bytes encryption_key = 13; + optional bool multiTenant = 14; + optional int32 tenantPrefixLength = 15; + optional uint64 section_index_offset = 16; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java index 9d99904e2131..ccb6172f6132 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java @@ -28,7 +28,10 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.hfile.BlockWithScanInfo; import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFileBlock; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; import org.apache.hadoop.hbase.io.hfile.HFileInfo; import org.apache.hadoop.hbase.io.hfile.HFileReaderImpl; import org.apache.hadoop.hbase.io.hfile.HFileScanner; @@ -356,31 +359,59 @@ public long getFilterEntries() { public void close(boolean evictOnClose) throws IOException { if (closed.compareAndSet(false, true)) { if (evictOnClose) { - final HFileReaderImpl.HFileScannerImpl s = - (HFileReaderImpl.HFileScannerImpl) super.getScanner(false, true, false); + long splitBlockOffset = findSplitBlockOffset(); final String reference = this.reader.getHFileInfo().getHFileContext().getHFileName(); final String referred = StoreFileInfo.getReferredToRegionAndFile(reference).getSecond(); - s.seekTo(splitCell); - if (s.getCurBlock() != null) { - long offset = s.getCurBlock().getOffset(); + if (splitBlockOffset >= 0) { LOG.trace("Seeking to split cell in reader: {} for file: {} top: {}, split offset: {}", - this, reference, top, offset); + this, reference, top, splitBlockOffset); ((HFileReaderImpl) reader).getCacheConf().getBlockCache().ifPresent(cache -> { int numEvictedReferred = top - ? cache.evictBlocksRangeByHfileName(referred, offset, Long.MAX_VALUE) - : cache.evictBlocksRangeByHfileName(referred, 0, offset); + ? cache.evictBlocksRangeByHfileName(referred, splitBlockOffset, Long.MAX_VALUE) + : cache.evictBlocksRangeByHfileName(referred, 0, splitBlockOffset); int numEvictedReference = cache.evictBlocksByHfileName(reference); LOG.trace( "Closing reference: {}; referred file: {}; was top? {}; evicted for referred: {};" + "evicted for reference: {}", reference, referred, top, numEvictedReferred, numEvictedReference); }); + } else { + LOG.debug("Unable to determine split block offset for reference {} (top? {})", reference, + top); } - s.close(); reader.close(false); } else { reader.close(evictOnClose); } } } + + private long findSplitBlockOffset() throws IOException { + HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader(); + if (indexReader != null) { + BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(splitCell, null, + false, true, false, reader.getEffectiveEncodingInCache(false), reader); + if (blockWithScanInfo != null) { + HFileBlock block = blockWithScanInfo.getHFileBlock(); + if (block != null) { + try { + return block.getOffset(); + } finally { + block.release(); + } + } + } + } + + try (HFileScanner scanner = super.getScanner(false, true, false)) { + if (scanner instanceof HFileReaderImpl.HFileScannerImpl) { + HFileReaderImpl.HFileScannerImpl delegate = (HFileReaderImpl.HFileScannerImpl) scanner; + delegate.seekTo(splitCell); + if (delegate.getCurBlock() != null) { + return delegate.getCurBlock().getOffset(); + } + } + } + return -1L; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MultiTenantFSDataInputStreamWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MultiTenantFSDataInputStreamWrapper.java new file mode 100644 index 000000000000..dd2af4c61ad5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MultiTenantFSDataInputStreamWrapper.java @@ -0,0 +1,450 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io; + +import java.io.IOException; +import java.io.InputStream; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PositionedReadable; +import org.apache.hadoop.fs.Seekable; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of {@link FSDataInputStreamWrapper} that adds offset translation capability for + * multi-tenant HFiles. This allows each tenant section to have its own coordinate system starting + * from 0, while the actual file positions are calculated by adding the section offset. + *

+ * The class transparently handles all position-related operations including: + *

+ */ +@InterfaceAudience.Private +public class MultiTenantFSDataInputStreamWrapper extends FSDataInputStreamWrapper { + private static final Logger LOG = + LoggerFactory.getLogger(MultiTenantFSDataInputStreamWrapper.class); + + /** The offset where this section starts in the parent file */ + private final long sectionOffset; + /** The original input stream wrapper to delegate to */ + private final FSDataInputStreamWrapper parent; + + /** + * Constructor that creates a wrapper with offset translation. + * @param parent the original input stream wrapper to delegate to + * @param offset the offset where the section starts in the parent file + */ + public MultiTenantFSDataInputStreamWrapper(FSDataInputStreamWrapper parent, long offset) { + // Use test constructor to properly initialize both streams and avoid assertion issues + super(parent.getStream(false), parent.getStream(true)); + this.parent = parent; + this.sectionOffset = offset; + + LOG.debug("Created section wrapper for section at offset {} (translation: {})", offset, + offset == 0 ? "none" : "enabled"); + } + + /** + * Converts a position relative to the section to an absolute file position. + * @param relativePosition the position relative to the section start + * @return the absolute position in the file + */ + public long toAbsolutePosition(long relativePosition) { + return relativePosition + sectionOffset; + } + + /** + * Converts an absolute file position to a position relative to the section. + * @param absolutePosition the absolute position in the file + * @return the position relative to the section start + */ + public long toRelativePosition(long absolutePosition) { + return absolutePosition - sectionOffset; + } + + @Override + public FSDataInputStream getStream(boolean useHBaseChecksum) { + // For all sections, wrap the raw stream with position translator + FSDataInputStream rawStream = parent.getStream(useHBaseChecksum); + return new TranslatingFSStream(rawStream); + } + + @Override + public Path getReaderPath() { + return parent.getReaderPath(); + } + + @Override + public boolean shouldUseHBaseChecksum() { + return parent.shouldUseHBaseChecksum(); + } + + @Override + public void prepareForBlockReader(boolean forceNoHBaseChecksum) throws IOException { + // Since we're using test constructor with hfs=null, prepareForBlockReader should return early + // and never hit the assertion. Call super instead of parent to avoid multiple calls on parent. + super.prepareForBlockReader(forceNoHBaseChecksum); + } + + @Override + public FSDataInputStream fallbackToFsChecksum(int offCount) throws IOException { + return parent.fallbackToFsChecksum(offCount); + } + + @Override + public void checksumOk() { + parent.checksumOk(); + } + + @Override + public void unbuffer() { + parent.unbuffer(); + } + + @Override + public void close() { + // Keep parent.close() behavior (do not close parent stream here) + } + + /** + * Custom implementation to translate seek position. + * @param seekPosition the position to seek to (section-relative) + * @throws IOException if an I/O error occurs + */ + public void seek(long seekPosition) throws IOException { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + + // Convert section-relative position to absolute file position + long absolutePosition = toAbsolutePosition(seekPosition); + stream.seek(absolutePosition); + } + + /** + * Custom implementation to translate position. + * @return the current position (section-relative) + * @throws IOException if an I/O error occurs + */ + public long getPos() throws IOException { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + long absolutePosition = stream.getPos(); + + // Get the absolute position and convert to section-relative position + return toRelativePosition(absolutePosition); + } + + /** + * Read method that translates position. + * @param buffer the buffer to read into + * @param offset the offset in the buffer + * @param length the number of bytes to read + * @return the number of bytes read + * @throws IOException if an I/O error occurs + */ + public int read(byte[] buffer, int offset, int length) throws IOException { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + return stream.read(buffer, offset, length); + } + + /** + * Custom implementation to read at position with offset translation. + * @param position the position to read from (section-relative) + * @param buffer the buffer to read into + * @param offset the offset in the buffer + * @param length the number of bytes to read + * @return the number of bytes read + * @throws IOException if an I/O error occurs + */ + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + + // Convert section-relative position to absolute file position + long absolutePosition = toAbsolutePosition(position); + return stream.read(absolutePosition, buffer, offset, length); + } + + /** + * Get the positioned readable interface. + * @return the positioned readable interface + */ + public PositionedReadable getPositionedReadable() { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + return stream; + } + + /** + * Get the seekable interface. + * @return the seekable interface + */ + public Seekable getSeekable() { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + return stream; + } + + /** + * Get the input stream. + * @return the input stream + */ + public InputStream getStream() { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + return stream; + } + + /** + * Check if an input stream is available. + * @return true if an input stream is available + */ + public boolean hasInputStream() { + return true; + } + + /** + * Check if positioned readable interface is available. + * @return true if positioned readable is available + */ + public boolean hasPositionedReadable() { + return true; + } + + /** + * Check if seekable interface is available. + * @return true if seekable is available + */ + public boolean hasSeekable() { + return true; + } + + /** + * Read a single byte. + * @return the byte read, or -1 if end of stream + * @throws IOException if an I/O error occurs + */ + public int read() throws IOException { + FSDataInputStream stream = parent.getStream(shouldUseHBaseChecksum()); + return stream.read(); + } + + /** + * Get the stream wrapper for the given stream. + * @param stream the stream to wrap + * @return the wrapped stream + */ + public FSDataInputStream getStream(FSDataInputStream stream) { + return stream; + } + + /** + * Translates section-relative seeks/reads into absolute file positions. + */ + private class TranslatingFSStream extends FSDataInputStream { + /** The raw underlying stream */ + private final FSDataInputStream rawStream; + + /** + * Constructor for TranslatingFSStream. + * @param rawStream the raw stream to wrap + */ + TranslatingFSStream(FSDataInputStream rawStream) { + super(new OffsetTranslatingInputStream(rawStream, sectionOffset)); + this.rawStream = rawStream; + // DO NOT automatically seek to sectionOffset here! + // This interferes with normal HFile reading patterns. + // The HFileReaderImpl will seek to specific positions as needed, + // and our translator will handle the offset translation. + LOG.debug("Created section stream wrapper for section starting at offset {}", sectionOffset); + } + + @Override + public void seek(long position) throws IOException { + // Convert section-relative position to absolute file position + long absolutePosition = toAbsolutePosition(position); + LOG.debug("Section seek: relative pos {} -> absolute pos {}, sectionOffset={}", position, + absolutePosition, sectionOffset); + // Validate that we're not seeking beyond reasonable bounds + if (position < 0) { + LOG.warn("Attempting to seek to negative relative position: {}", position); + } + rawStream.seek(absolutePosition); + } + + @Override + public long getPos() throws IOException { + long absolutePosition = rawStream.getPos(); + + // Convert absolute position to section-relative position + long relativePosition = toRelativePosition(absolutePosition); + LOG.trace("Section getPos: absolute {} -> relative {}, sectionOffset={}", absolutePosition, + relativePosition, sectionOffset); + // Validate position translation + if (relativePosition < 0) { + LOG.warn( + "Position translation resulted in negative relative position: " + + "absolute={}, relative={}, sectionOffset={}", + absolutePosition, relativePosition, sectionOffset); + } + return relativePosition; + } + + @Override + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + // Convert section-relative position to absolute file position + long absolutePosition = toAbsolutePosition(position); + LOG.trace("Section pread: relative pos {} -> absolute pos {}, len={}, sectionOffset={}", + position, absolutePosition, length, sectionOffset); + // Validate read parameters + if (position < 0) { + LOG.warn("Attempting to read from negative relative position: {}", position); + } + if (length < 0) { + throw new IllegalArgumentException("Read length cannot be negative: " + length); + } + return rawStream.read(absolutePosition, buffer, offset, length); + } + + @Override + public boolean seekToNewSource(long targetPosition) throws IOException { + return rawStream.seekToNewSource(toAbsolutePosition(targetPosition)); + } + + // Other read methods use the underlying stream's implementations + // Note: We cannot override final methods like read(), read(byte[]), etc. + } + + /** + * Custom InputStream that translates all read operations by adding the section offset. This + * ensures that when DataInputStream's final methods call read(), they go through our offset + * translation logic. + */ + private static class OffsetTranslatingInputStream extends InputStream + implements Seekable, PositionedReadable { + /** The raw underlying stream */ + private final FSDataInputStream rawStream; + /** The section offset for translation */ + private final long sectionOffset; + + /** + * Constructor for OffsetTranslatingInputStream. + * @param rawStream the raw stream to wrap + * @param sectionOffset the section offset for translation + */ + OffsetTranslatingInputStream(FSDataInputStream rawStream, long sectionOffset) { + this.rawStream = rawStream; + this.sectionOffset = sectionOffset; + } + + @Override + public int read() throws IOException { + // For single byte reads, we rely on the current position being correctly set + return rawStream.read(); + } + + @Override + public int read(byte[] buffer, int offset, int length) throws IOException { + // This is the key method that gets called by DataInputStream's final methods + // We need to ensure the stream is positioned correctly before reading + return rawStream.read(buffer, offset, length); + } + + @Override + public long skip(long bytesToSkip) throws IOException { + return rawStream.skip(bytesToSkip); + } + + @Override + public int available() throws IOException { + return rawStream.available(); + } + + @Override + public void close() throws IOException { + rawStream.close(); + } + + @Override + public void mark(int readLimit) { + rawStream.mark(readLimit); + } + + @Override + public void reset() throws IOException { + rawStream.reset(); + } + + @Override + public boolean markSupported() { + return rawStream.markSupported(); + } + + // Seekable interface implementation + @Override + public void seek(long position) throws IOException { + // Translate section-relative position to absolute file position + long absolutePosition = sectionOffset + position; + LOG.trace("OffsetTranslatingInputStream seek: relative pos {} -> absolute pos {}, " + + "sectionOffset={}", position, absolutePosition, sectionOffset); + rawStream.seek(absolutePosition); + } + + @Override + public long getPos() throws IOException { + // Translate absolute file position back to section-relative position + long absolutePosition = rawStream.getPos(); + long relativePosition = absolutePosition - sectionOffset; + LOG.trace("OffsetTranslatingInputStream getPos: absolute pos {} -> relative pos {}, " + + "sectionOffset={}", absolutePosition, relativePosition, sectionOffset); + return relativePosition; + } + + @Override + public boolean seekToNewSource(long targetPosition) throws IOException { + // Translate section-relative position to absolute file position + long absolutePosition = sectionOffset + targetPosition; + LOG.trace("OffsetTranslatingInputStream seekToNewSource: relative pos {} -> " + + "absolute pos {}, sectionOffset={}", targetPosition, absolutePosition, sectionOffset); + return rawStream.seekToNewSource(absolutePosition); + } + + // PositionedReadable interface implementation + @Override + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + // Translate section-relative position to absolute file position + long absolutePosition = sectionOffset + position; + LOG.trace("OffsetTranslatingInputStream pread: relative pos {} -> absolute pos {}, " + + "len={}, sectionOffset={}", position, absolutePosition, length, sectionOffset); + return rawStream.read(absolutePosition, buffer, offset, length); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { + // Translate section-relative position to absolute file position + long absolutePosition = sectionOffset + position; + LOG.trace("OffsetTranslatingInputStream readFully: relative pos {} -> absolute pos {}, " + + "len={}, sectionOffset={}", position, absolutePosition, length, sectionOffset); + rawStream.readFully(absolutePosition, buffer, offset, length); + } + + @Override + public void readFully(long position, byte[] buffer) throws IOException { + readFully(position, buffer, 0, buffer.length); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractMultiTenantReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractMultiTenantReader.java new file mode 100644 index 000000000000..51731c57e092 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractMultiTenantReader.java @@ -0,0 +1,2469 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.io.MultiTenantFSDataInputStreamWrapper; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.util.BloomFilter; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.cache.Cache; +import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.hbase.thirdparty.com.google.common.cache.RemovalNotification; + +/** + * Abstract base class for multi-tenant HFile readers. This class handles the common functionality + * for both pread and stream access modes, delegating specific reader creation to subclasses. + *

+ * The multi-tenant reader acts as a router that: + *

    + *
  1. Extracts tenant information from cell keys
  2. + *
  3. Locates the appropriate section in the HFile for that tenant
  4. + *
  5. Delegates reading operations to a standard v3 reader for that section
  6. + *
+ *

+ * Key features: + *

+ */ +@InterfaceAudience.Private +public abstract class AbstractMultiTenantReader extends HFileReaderImpl + implements MultiTenantBloomSupport { + private static final Logger LOG = LoggerFactory.getLogger(AbstractMultiTenantReader.class); + + /** Static storage for table properties to avoid repeated loading */ + private static final Cache> TABLE_PROPERTIES_CACHE = + CacheBuilder.newBuilder().maximumSize(100).expireAfterWrite(5, TimeUnit.MINUTES).build(); + + /** Tenant extractor for identifying tenant information from cells */ + protected final TenantExtractor tenantExtractor; + /** Section index reader for locating tenant sections */ + protected final SectionIndexManager.Reader sectionIndexReader; + + /** Configuration key for section prefetch enablement */ + private static final String SECTION_PREFETCH_ENABLED = + "hbase.multi.tenant.reader.prefetch.enabled"; + /** Default prefetch enabled flag */ + private static final boolean DEFAULT_SECTION_PREFETCH_ENABLED = true; + + /** Configuration key controlling meta block lookup caching */ + private static final String META_BLOCK_CACHE_ENABLED = + "hbase.multi.tenant.reader.meta.cache.enabled"; + /** Default flag enabling meta block cache */ + private static final boolean DEFAULT_META_BLOCK_CACHE_ENABLED = true; + /** Configuration key controlling cache size */ + private static final String META_BLOCK_CACHE_MAX_SIZE = + "hbase.multi.tenant.reader.meta.cache.max"; + /** Default maximum cache entries */ + private static final int DEFAULT_META_BLOCK_CACHE_MAX_SIZE = 256; + + /** Configuration key controlling data block index lookup caching */ + private static final String DATA_BLOCK_INDEX_CACHE_ENABLED = + "hbase.multi.tenant.reader.data.index.cache.enabled"; + /** Default flag enabling data block index cache */ + private static final boolean DEFAULT_DATA_BLOCK_INDEX_CACHE_ENABLED = true; + + /** Configuration key for the maximum number of cached section readers */ + private static final String SECTION_READER_CACHE_MAX_SIZE = + "hbase.multi.tenant.reader.section.cache.max"; + /** Default maximum number of cached section readers */ + private static final int DEFAULT_SECTION_READER_CACHE_MAX_SIZE = 8; + /** Configuration key for section reader cache idle eviction (ms) */ + private static final String SECTION_READER_CACHE_EXPIRE_MS = + "hbase.multi.tenant.reader.section.cache.expire.ms"; + /** Default idle eviction (1 minute) */ + private static final long DEFAULT_SECTION_READER_CACHE_EXPIRE_MS = TimeUnit.MINUTES.toMillis(1); + + /** Private map to store section metadata */ + private final Map sectionLocations = + new LinkedHashMap(); + + /** List for section navigation */ + private List sectionIds; + + /** Number of levels in the tenant index structure */ + private int tenantIndexLevels = 1; + /** Maximum chunk size used in the tenant index */ + private int tenantIndexMaxChunkSize = SectionIndexManager.DEFAULT_MAX_CHUNK_SIZE; + /** Whether prefetch is enabled for sequential access */ + private final boolean prefetchEnabled; + /** Cache of section readers keyed by tenant section ID */ + private final Cache sectionReaderCache; + /** Cached Bloom filter state per section */ + private final Cache sectionBloomCache; + private final AtomicReference generalBloomTypeCache = new AtomicReference<>(); + /** Whether we cache meta block to section mappings */ + private final boolean metaBlockCacheEnabled; + /** Cache for meta block name to section mapping */ + private final Cache metaBlockSectionCache; + /** Whether we cache data block index section hints */ + private final boolean dataBlockIndexCacheEnabled; + /** Cached section hint for data block index lookup */ + private final AtomicReference dataBlockIndexSectionHint; + + /** + * Constructor for multi-tenant reader. + * @param context Reader context info + * @param fileInfo HFile info + * @param cacheConf Cache configuration values + * @param conf Configuration + * @throws IOException If an error occurs during initialization + */ + public AbstractMultiTenantReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf, + Configuration conf) throws IOException { + super(context, fileInfo, cacheConf, conf); + + // Initialize section index reader + this.sectionIndexReader = new SectionIndexManager.Reader(); + + // Load tenant index metadata before accessing the section index so we know how to interpret it + loadTenantIndexMetadata(); + + // Initialize section index using dataBlockIndexReader from parent + initializeSectionIndex(); + + // Log tenant index structure information once sections are available + logTenantIndexStructureInfo(); + + // Create tenant extractor with consistent configuration + this.tenantExtractor = TenantExtractorFactory.createFromReader(this); + + // Initialize prefetch configuration + this.prefetchEnabled = + conf.getBoolean(SECTION_PREFETCH_ENABLED, DEFAULT_SECTION_PREFETCH_ENABLED); + + // Initialize cache for section readers + this.sectionReaderCache = createSectionReaderCache(conf); + this.sectionBloomCache = createSectionBloomCache(conf); + + this.metaBlockCacheEnabled = + conf.getBoolean(META_BLOCK_CACHE_ENABLED, DEFAULT_META_BLOCK_CACHE_ENABLED); + if (metaBlockCacheEnabled) { + int maxEntries = conf.getInt(META_BLOCK_CACHE_MAX_SIZE, DEFAULT_META_BLOCK_CACHE_MAX_SIZE); + this.metaBlockSectionCache = CacheBuilder.newBuilder().maximumSize(maxEntries).build(); + } else { + this.metaBlockSectionCache = null; + } + + this.dataBlockIndexCacheEnabled = + conf.getBoolean(DATA_BLOCK_INDEX_CACHE_ENABLED, DEFAULT_DATA_BLOCK_INDEX_CACHE_ENABLED); + this.dataBlockIndexSectionHint = new AtomicReference<>(); + + LOG.info("Initialized multi-tenant reader for {}", context.getFilePath()); + } + + /** + * Initialize the section index from the file. + * @throws IOException If an error occurs loading the section index + */ + protected void initializeSectionIndex() throws IOException { + // Get the trailer directly + FixedFileTrailer trailer = fileInfo.getTrailer(); + + // Access the input stream through the context + FSDataInputStreamWrapper fsWrapper = context.getInputStreamWrapper(); + FSDataInputStream fsdis = fsWrapper.getStream(fsWrapper.shouldUseHBaseChecksum()); + long originalPosition = fsdis.getPos(); + + long sectionIndexOffset = getSectionIndexOffset(); + try { + LOG.debug("Seeking to load-on-open section at offset {}", sectionIndexOffset); + + // Fall back to trailer-provided level count when metadata has not been loaded yet + int trailerLevels = trailer.getNumDataIndexLevels(); + if (trailerLevels >= 1 && trailerLevels > tenantIndexLevels) { + tenantIndexLevels = trailerLevels; + } + + // In HFile v4, the tenant index is stored at the recorded section index offset + HFileBlock rootIndexBlock = + getUncachedBlockReader().readBlockData(sectionIndexOffset, -1, true, false, false); + + // Validate this is a root index block + if (rootIndexBlock.getBlockType() != BlockType.ROOT_INDEX) { + throw new IOException("Expected ROOT_INDEX block for tenant index in HFile v4, found " + + rootIndexBlock.getBlockType() + " at offset " + trailer.getLoadOnOpenDataOffset()); + } + + HFileBlock blockToRead = null; + try { + blockToRead = rootIndexBlock.unpack(getFileContext(), getUncachedBlockReader()); + + // Load the section index from the root block (support multi-level traversal) + if (tenantIndexLevels <= 1) { + sectionIndexReader.loadSectionIndex(blockToRead); + } else { + sectionIndexReader.loadSectionIndex(blockToRead, tenantIndexLevels, + getUncachedBlockReader()); + } + + // Copy section info to our internal data structures + initSectionLocations(); + + LOG.debug("Initialized tenant section index with {} entries", getSectionCount()); + } finally { + if (blockToRead != null) { + blockToRead.release(); + if (blockToRead != rootIndexBlock) { + rootIndexBlock.release(); + } + } else { + rootIndexBlock.release(); + } + } + } catch (IOException e) { + LOG.error("Failed to load tenant section index", e); + throw e; + } finally { + // Restore original position + fsdis.seek(originalPosition); + } + } + + private long getSectionIndexOffset() { + long offset = trailer.getSectionIndexOffset(); + return offset >= 0 ? offset : trailer.getLoadOnOpenDataOffset(); + } + + /** + * Load information about the tenant index structure from file info. + *

+ * Extracts tenant index levels and chunk size configuration from the HFile metadata to optimize + * section lookup performance. + */ + private void logTenantIndexStructureInfo() { + int numSections = getSectionCount(); + if (tenantIndexLevels > 1) { + LOG.info("Multi-tenant HFile loaded with {} sections using {}-level tenant index " + + "(maxChunkSize={})", numSections, tenantIndexLevels, tenantIndexMaxChunkSize); + } else { + LOG.info("Multi-tenant HFile loaded with {} sections using single-level tenant index", + numSections); + } + + LOG.debug("Tenant index details: levels={}, chunkSize={}, sections={}", tenantIndexLevels, + tenantIndexMaxChunkSize, numSections); + } + + private Cache + createSectionReaderCache(Configuration conf) { + int maxSize = Math.max(1, + conf.getInt(SECTION_READER_CACHE_MAX_SIZE, DEFAULT_SECTION_READER_CACHE_MAX_SIZE)); + long expireMs = + conf.getLong(SECTION_READER_CACHE_EXPIRE_MS, DEFAULT_SECTION_READER_CACHE_EXPIRE_MS); + + CacheBuilder builder = CacheBuilder.newBuilder().maximumSize(maxSize); + if (expireMs > 0) { + builder.expireAfterAccess(expireMs, TimeUnit.MILLISECONDS); + } + + Cache cache = builder.removalListener( + (RemovalNotification notification) -> { + SectionReaderHolder holder = notification.getValue(); + if (holder != null) { + holder.markEvicted(true); + } + }).build(); + LOG.debug("Initialized section reader cache with maxSize={}, expireMs={}", maxSize, expireMs); + return cache; + } + + private Cache + createSectionBloomCache(Configuration conf) { + int maxSize = Math.max(1, + conf.getInt(SECTION_READER_CACHE_MAX_SIZE, DEFAULT_SECTION_READER_CACHE_MAX_SIZE)); + Cache cache = + CacheBuilder.newBuilder().maximumSize(maxSize).build(); + LOG.debug("Initialized section bloom cache with maxSize={}", maxSize); + return cache; + } + + private void loadTenantIndexMetadata() { + byte[] tenantIndexLevelsBytes = + fileInfo.get(Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_TENANT_INDEX_LEVELS)); + if (tenantIndexLevelsBytes != null) { + int parsedLevels = Bytes.toInt(tenantIndexLevelsBytes); + if (parsedLevels >= 1) { + tenantIndexLevels = parsedLevels; + } else { + LOG.warn("Ignoring invalid tenant index level count {} in file info for {}", parsedLevels, + context.getFilePath()); + tenantIndexLevels = 1; + } + } + + byte[] chunkSizeBytes = + fileInfo.get(Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_TENANT_INDEX_MAX_CHUNK)); + if (chunkSizeBytes != null) { + int parsedChunkSize = Bytes.toInt(chunkSizeBytes); + if (parsedChunkSize > 0) { + tenantIndexMaxChunkSize = parsedChunkSize; + } else { + LOG.warn("Ignoring invalid tenant index chunk size {} in file info for {}", parsedChunkSize, + context.getFilePath()); + } + } + } + + /** + * Get the number of levels in the tenant index. + * @return The number of levels (1 for single-level, 2+ for multi-level) + */ + public int getTenantIndexLevels() { + return tenantIndexLevels; + } + + /** + * Get the maximum chunk size used in the tenant index. + * @return The maximum entries per index block + */ + public int getTenantIndexMaxChunkSize() { + return tenantIndexMaxChunkSize; + } + + /** + * Initialize our section location map from the index reader. + *

+ * Populates the internal section metadata map and creates the section ID list for efficient + * navigation during scanning operations. + */ + private void initSectionLocations() { + for (SectionIndexManager.SectionIndexEntry entry : sectionIndexReader.getSections()) { + sectionLocations.put(new ImmutableBytesWritable(entry.getTenantPrefix()), + new SectionMetadata(entry.getOffset(), entry.getSectionSize())); + } + + // Create list for section navigation + sectionIds = new ArrayList<>(sectionLocations.keySet()); + // Sort by tenant prefix to ensure lexicographic order + sectionIds.sort((a, b) -> Bytes.compareTo(a.get(), b.get())); + LOG.debug("Initialized {} section IDs for navigation", sectionIds.size()); + } + + /** + * Get the number of sections in this file. + * @return The number of sections in this file + */ + private int getSectionCount() { + return sectionLocations.size(); + } + + /** + * Get the total number of tenant sections in this file. + * @return The number of sections + */ + public int getTotalSectionCount() { + return sectionLocations.size(); + } + + @Override + public boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) + throws IOException { + byte[] sectionId = extractTenantSectionId(row, rowOffset, rowLen); + if (sectionId == null) { + return true; + } + ImmutableBytesWritable cacheKey = new ImmutableBytesWritable(sectionId); + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + return true; + } + try (lease) { + SectionBloomState bloomState = getOrLoadSectionBloomState(cacheKey, lease); + if (bloomState == null || !bloomState.hasGeneralBloom()) { + return true; + } + return bloomState.passesGeneralRowBloom(row, rowOffset, rowLen, lease.getReader()); + } + } + + @Override + public boolean passesGeneralRowColBloomFilter(ExtendedCell cell) throws IOException { + byte[] sectionId = tenantExtractor.extractTenantSectionId(cell); + if (sectionId == null) { + return true; + } + ImmutableBytesWritable cacheKey = new ImmutableBytesWritable(sectionId); + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + return true; + } + try (lease) { + SectionBloomState bloomState = getOrLoadSectionBloomState(cacheKey, lease); + if (bloomState == null || !bloomState.hasGeneralBloom()) { + return true; + } + return bloomState.passesGeneralRowColBloom(cell, lease.getReader()); + } + } + + @Override + public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowLen) + throws IOException { + byte[] sectionId = extractTenantSectionId(row, rowOffset, rowLen); + if (sectionId == null) { + return true; + } + ImmutableBytesWritable cacheKey = new ImmutableBytesWritable(sectionId); + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + return true; + } + try (lease) { + SectionBloomState bloomState = getOrLoadSectionBloomState(cacheKey, lease); + if (bloomState == null) { + return true; + } + if (bloomState.getDeleteFamilyCnt() == 0) { + return false; + } + if (!bloomState.hasDeleteFamilyBloom()) { + return true; + } + return bloomState.passesDeleteFamilyBloom(row, rowOffset, rowLen); + } + } + + private byte[] extractTenantSectionId(byte[] row, int rowOffset, int rowLen) { + if (row == null) { + return null; + } + int prefixLength = tenantExtractor.getPrefixLength(); + if (prefixLength > 0 && rowLen < prefixLength) { + return null; + } + ExtendedCell lookupCell = PrivateCellUtil.createFirstOnRow(row, rowOffset, (short) rowLen); + return tenantExtractor.extractTenantSectionId(lookupCell); + } + + private SectionBloomState getOrLoadSectionBloomState(ImmutableBytesWritable cacheKey, + SectionReaderLease lease) throws IOException { + try { + return sectionBloomCache.get(cacheKey, () -> loadSectionBloomState(cacheKey, lease)); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw (IOException) cause; + } + throw new IOException( + "Failed to load bloom state for section " + + Bytes.toStringBinary(cacheKey.get(), cacheKey.getOffset(), cacheKey.getLength()), + cause); + } + } + + private SectionBloomState loadSectionBloomState(ImmutableBytesWritable sectionKey, + SectionReaderLease lease) throws IOException { + HFileReaderImpl reader = lease.getReader(); + Map fileInfoMap = reader.getHFileInfo(); + + BloomType bloomType = BloomType.NONE; + byte[] bloomTypeBytes = fileInfoMap.get(HStoreFile.BLOOM_FILTER_TYPE_KEY); + if (bloomTypeBytes != null) { + bloomType = BloomType.valueOf(Bytes.toString(bloomTypeBytes)); + } + + BloomFilter generalBloom = null; + DataInput generalMeta = reader.getGeneralBloomFilterMetadata(); + if (generalMeta != null) { + generalBloom = BloomFilterFactory.createFromMeta(generalMeta, reader, null); + } + + BloomFilter deleteBloom = null; + DataInput deleteMeta = reader.getDeleteBloomFilterMetadata(); + if (deleteMeta != null) { + deleteBloom = BloomFilterFactory.createFromMeta(deleteMeta, reader, null); + } + + byte[] lastBloomKey = fileInfoMap.get(HStoreFile.LAST_BLOOM_KEY); + KeyValue.KeyOnlyKeyValue lastBloomKeyKV = null; + if (lastBloomKey != null && bloomType == BloomType.ROWCOL) { + lastBloomKeyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length); + } + + int prefixLength = 0; + byte[] bloomParam = fileInfoMap.get(HStoreFile.BLOOM_FILTER_PARAM_KEY); + if (bloomParam != null) { + prefixLength = Bytes.toInt(bloomParam); + } + + long deleteFamilyCnt = 0L; + byte[] deleteFamilyCntBytes = fileInfoMap.get(HStoreFile.DELETE_FAMILY_COUNT); + if (deleteFamilyCntBytes != null) { + deleteFamilyCnt = Bytes.toLong(deleteFamilyCntBytes); + } + + long entries = reader.getTrailer().getEntryCount(); + + return new SectionBloomState(sectionKey.copyBytes(), bloomType, generalBloom, deleteBloom, + lastBloomKey, lastBloomKeyKV, prefixLength, deleteFamilyCnt, entries, reader.getComparator()); + } + + private SectionBloomState findSectionBloomState(boolean needGeneral, boolean needDelete) + throws IOException { + if (sectionIds == null || sectionIds.isEmpty()) { + return null; + } + for (ImmutableBytesWritable sectionId : sectionIds) { + byte[] key = sectionId.copyBytes(); + SectionReaderLease lease = getSectionReader(key); + if (lease == null) { + continue; + } + try (lease) { + SectionBloomState state = getOrLoadSectionBloomState(sectionId, lease); + if (state == null) { + continue; + } + if (needGeneral && !state.hasGeneralBloom()) { + continue; + } + if (needDelete && !state.hasDeleteFamilyBloom()) { + continue; + } + return state; + } + } + return null; + } + + @Override + public BloomType getGeneralBloomFilterType() { + BloomType cached = generalBloomTypeCache.get(); + if (cached != null) { + return cached; + } + + BloomType computed = BloomType.NONE; + try { + SectionBloomState state = findSectionBloomState(true, false); + if (state != null && state.hasGeneralBloom()) { + computed = state.getBloomType(); + } + } catch (IOException e) { + LOG.debug("Failed to inspect bloom type", e); + } + + generalBloomTypeCache.compareAndSet(null, computed); + BloomType result = generalBloomTypeCache.get(); + return result != null ? result : computed; + } + + @Override + public int getGeneralBloomPrefixLength() throws IOException { + SectionBloomState state = findSectionBloomState(true, false); + return state != null ? state.getPrefixLength() : 0; + } + + @Override + public byte[] getLastBloomKey() throws IOException { + SectionBloomState state = findSectionBloomState(true, false); + return state != null ? state.getLastBloomKey() : null; + } + + @Override + public long getDeleteFamilyBloomCount() throws IOException { + SectionBloomState state = findSectionBloomState(false, true); + return state != null ? state.getDeleteFamilyCnt() : 0L; + } + + @Override + public BloomFilter getGeneralBloomFilterInstance() throws IOException { + SectionBloomState state = findSectionBloomState(true, false); + return state != null ? state.getGeneralBloom() : null; + } + + @Override + public BloomFilter getDeleteFamilyBloomFilterInstance() throws IOException { + SectionBloomState state = findSectionBloomState(false, true); + return state != null ? state.getDeleteBloom() : null; + } + + /** + * Get table properties from the file context if available. + *

+ * Properties are used for tenant configuration and optimization settings. + * @return A map of table properties, or empty map if not available + */ + protected Map getTableProperties() { + Map tableProperties = new HashMap<>(); + + try { + // If file context has table name, try to get table properties + HFileContext fileContext = getFileContext(); + if (fileContext == null || fileContext.getTableName() == null) { + LOG.debug("Table name not available in HFileContext"); + return tableProperties; + } + + // Get the table descriptor from Admin API + TableName tableName = TableName.valueOf(fileContext.getTableName()); + + try { + tableProperties = TABLE_PROPERTIES_CACHE.get(tableName, () -> { + Map props = new HashMap<>(); + try (Connection conn = ConnectionFactory.createConnection(getConf()); + Admin admin = conn.getAdmin()) { + TableDescriptor tableDesc = admin.getDescriptor(tableName); + if (tableDesc != null) { + // Extract relevant properties for multi-tenant configuration + tableDesc.getValues().forEach((k, v) -> { + props.put(Bytes.toString(k.get()), Bytes.toString(v.get())); + }); + LOG.debug("Loaded table properties for {}", tableName); + } + } + return props; + }); + } catch (Exception e) { + LOG.warn("Failed to get table descriptor for {}", tableName, e); + } + } catch (Exception e) { + LOG.warn("Error loading table properties", e); + } + + return tableProperties; + } + + /** + * Metadata for a tenant section within the HFile. + */ + protected static class SectionMetadata { + /** The offset where the section starts */ + final long offset; + /** The size of the section in bytes */ + final int size; + + /** + * Constructor for SectionMetadata. + * @param offset the file offset where the section starts + * @param size the size of the section in bytes + */ + SectionMetadata(long offset, int size) { + this.offset = offset; + this.size = size; + } + + /** + * Get the offset where the section starts. + * @return the section offset + */ + long getOffset() { + return offset; + } + + /** + * Get the size of the section. + * @return the section size in bytes + */ + int getSize() { + return size; + } + } + + /** + * Get metadata for a tenant section. + * @param tenantSectionId The tenant section ID to look up + * @return Section metadata or null if not found + * @throws IOException If an error occurs during lookup + */ + protected SectionMetadata getSectionMetadata(byte[] tenantSectionId) throws IOException { + return sectionLocations.get(new ImmutableBytesWritable(tenantSectionId)); + } + + /** + * Create a reader for a tenant section on demand. + * @param tenantSectionId The tenant section ID for the section + * @return A section reader or null if the section doesn't exist + * @throws IOException If an error occurs creating the reader + */ + protected SectionReaderLease getSectionReader(byte[] tenantSectionId) throws IOException { + SectionMetadata metadata = getSectionMetadata(tenantSectionId); + if (metadata == null) { + return null; + } + + final ImmutableBytesWritable cacheKey = + new ImmutableBytesWritable(tenantSectionId, 0, tenantSectionId.length); + final SectionMetadata sectionMetadata = metadata; + try { + SectionReaderHolder holder = sectionReaderCache.get(cacheKey, () -> { + byte[] sectionIdCopy = Bytes.copy(tenantSectionId); + SectionReader sectionReader = createSectionReader(sectionIdCopy, sectionMetadata); + return new SectionReaderHolder(sectionReader); + }); + holder.retain(); + return new SectionReaderLease(cacheKey, holder); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw (IOException) cause; + } + throw new IOException( + "Failed to create section reader for tenant " + Bytes.toStringBinary(tenantSectionId), + cause); + } + } + + /** + * Create appropriate section reader based on type (to be implemented by subclasses). + * @param tenantSectionId The tenant section ID + * @param metadata The section metadata + * @return A section reader + * @throws IOException If an error occurs creating the reader + */ + protected abstract SectionReader createSectionReader(byte[] tenantSectionId, + SectionMetadata metadata) throws IOException; + + /** + * Get a scanner for this file. + * @param conf Configuration to use + * @param cacheBlocks Whether to cache blocks + * @param pread Whether to use positional read + * @param isCompaction Whether this is for a compaction + * @return A scanner for this file + */ + @Override + public HFileScanner getScanner(Configuration conf, boolean cacheBlocks, boolean pread, + boolean isCompaction) { + return new MultiTenantScanner(conf, cacheBlocks, pread, isCompaction); + } + + /** + * Simpler scanner method that delegates to the full method. + * @param conf Configuration to use + * @param cacheBlocks Whether to cache blocks + * @param pread Whether to use positional read + * @return A scanner for this file + */ + @Override + public HFileScanner getScanner(Configuration conf, boolean cacheBlocks, boolean pread) { + return getScanner(conf, cacheBlocks, pread, false); + } + + /** + * Abstract base class for section readers. + *

+ * Each section reader manages access to a specific tenant section within the HFile, providing + * transparent delegation to standard HFile v3 readers with proper offset translation and resource + * management. + */ + protected abstract class SectionReader { + /** The tenant section ID for this reader */ + protected final byte[] tenantSectionId; + /** The section metadata */ + protected final SectionMetadata metadata; + /** The underlying HFile reader */ + protected HFileReaderImpl reader; + /** Whether this reader has been initialized */ + protected boolean initialized = false; + /** The base offset for this section */ + protected long sectionBaseOffset; + + /** + * Constructor for SectionReader. + * @param tenantSectionId The tenant section ID + * @param metadata The section metadata + */ + public SectionReader(byte[] tenantSectionId, SectionMetadata metadata) { + this.tenantSectionId = tenantSectionId.clone(); // Make defensive copy + this.metadata = metadata; + this.sectionBaseOffset = metadata.getOffset(); + } + + /** + * Get or initialize the underlying reader. + * @return The underlying HFile reader + * @throws IOException If an error occurs initializing the reader + */ + public abstract HFileReaderImpl getReader() throws IOException; + + /** + * Get a scanner for this section. + * @param conf Configuration to use + * @param cacheBlocks Whether to cache blocks + * @param pread Whether to use positional read + * @param isCompaction Whether this is for a compaction + * @return A scanner for this section + * @throws IOException If an error occurs creating the scanner + */ + public abstract HFileScanner getScanner(Configuration conf, boolean cacheBlocks, boolean pread, + boolean isCompaction) throws IOException; + + /** + * Close the section reader. + * @throws IOException If an error occurs closing the reader + */ + public void close() throws IOException { + close(false); + } + + /** + * Close the section reader. + * @param evictOnClose whether to evict blocks on close + * @throws IOException If an error occurs closing the reader + */ + public abstract void close(boolean evictOnClose) throws IOException; + } + + private static final class SectionBloomState { + private final byte[] sectionId; + private final BloomType bloomType; + private final BloomFilter generalBloom; + private final BloomFilter deleteBloom; + private final byte[] lastBloomKey; + private final KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV; + private final int prefixLength; + private final long deleteFamilyCnt; + private final long entryCount; + private final CellComparator comparator; + + SectionBloomState(byte[] sectionId, BloomType bloomType, BloomFilter generalBloom, + BloomFilter deleteBloom, byte[] lastBloomKey, KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV, + int prefixLength, long deleteFamilyCnt, long entryCount, CellComparator comparator) { + this.sectionId = sectionId; + this.bloomType = bloomType == null ? BloomType.NONE : bloomType; + this.generalBloom = generalBloom; + this.deleteBloom = deleteBloom; + this.lastBloomKey = lastBloomKey; + this.lastBloomKeyOnlyKV = lastBloomKeyOnlyKV; + this.prefixLength = prefixLength; + this.deleteFamilyCnt = deleteFamilyCnt; + this.entryCount = entryCount; + this.comparator = comparator; + } + + boolean hasGeneralBloom() { + return generalBloom != null && bloomType != BloomType.NONE; + } + + BloomType getBloomType() { + return bloomType; + } + + boolean hasDeleteFamilyBloom() { + return deleteBloom != null; + } + + boolean passesGeneralRowBloom(byte[] row, int rowOffset, int rowLen, HFileReaderImpl reader) + throws IOException { + if (!hasGeneralBloom()) { + return true; + } + if (entryCount == 0) { + return false; + } + if (bloomType == BloomType.ROWCOL) { + // Without column information we cannot make a definitive call. + return true; + } + int keyOffset = rowOffset; + int keyLen = rowLen; + if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) { + if (prefixLength <= 0 || rowLen < prefixLength) { + return true; + } + keyLen = prefixLength; + } + return checkGeneralBloomFilter(row, keyOffset, keyLen, null, reader); + } + + boolean passesGeneralRowColBloom(ExtendedCell cell, HFileReaderImpl reader) throws IOException { + if (!hasGeneralBloom()) { + return true; + } + if (entryCount == 0) { + return false; + } + if (bloomType != BloomType.ROWCOL) { + // When the Bloom filter is not a ROWCOL type, fall back to row-based filtering. + ExtendedCell rowCell = PrivateCellUtil.createFirstOnRow(cell); + byte[] rowKey = rowCell.getRowArray(); + return checkGeneralBloomFilter(rowKey, rowCell.getRowOffset(), rowCell.getRowLength(), null, + reader); + } + ExtendedCell kvKey = PrivateCellUtil.createFirstOnRowCol(cell); + return checkGeneralBloomFilter(null, 0, 0, kvKey, reader); + } + + boolean passesDeleteFamilyBloom(byte[] row, int rowOffset, int rowLen) { + if (deleteFamilyCnt == 0) { + return false; + } + if (deleteBloom == null) { + return true; + } + return deleteBloom.contains(row, rowOffset, rowLen, null); + } + + int getPrefixLength() { + return prefixLength; + } + + byte[] getLastBloomKey() { + return lastBloomKey != null ? lastBloomKey.clone() : null; + } + + long getDeleteFamilyCnt() { + return deleteFamilyCnt; + } + + BloomFilter getGeneralBloom() { + return generalBloom; + } + + BloomFilter getDeleteBloom() { + return deleteBloom; + } + + private boolean checkGeneralBloomFilter(byte[] key, int keyOffset, int keyLen, Cell kvKey, + HFileReaderImpl reader) throws IOException { + ByteBuff bloomData = null; + HFileBlock bloomBlock = null; + try { + if (!generalBloom.supportsAutoLoading()) { + bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true); + if (bloomBlock == null) { + return true; + } + bloomData = bloomBlock.getBufferWithoutHeader(); + } + + boolean keyIsAfterLast = false; + if (lastBloomKey != null) { + if (bloomType == BloomType.ROWCOL && kvKey != null && comparator != null) { + keyIsAfterLast = comparator.compare(kvKey, lastBloomKeyOnlyKV) > 0; + } else if (key != null) { + keyIsAfterLast = + Bytes.compareTo(key, keyOffset, keyLen, lastBloomKey, 0, lastBloomKey.length) > 0; + } + } + + if (bloomType == BloomType.ROWCOL && kvKey != null) { + ExtendedCell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey); + if ( + keyIsAfterLast && comparator != null + && comparator.compare(rowBloomKey, lastBloomKeyOnlyKV) > 0 + ) { + return false; + } + return generalBloom.contains(kvKey, bloomData, BloomType.ROWCOL) + || generalBloom.contains(rowBloomKey, bloomData, BloomType.ROWCOL); + } else { + if (keyIsAfterLast) { + return false; + } + return generalBloom.contains(key, keyOffset, keyLen, bloomData); + } + } finally { + if (bloomBlock != null) { + bloomBlock.release(); + } + } + } + + @Override + public String toString() { + return "SectionBloomState{" + Bytes.toStringBinary(sectionId) + ", type=" + bloomType + '}'; + } + } + + /** + * Cache entry wrapper managing lifecycle and reference counting for section readers. + */ + private static final class SectionReaderHolder { + private final SectionReader sectionReader; + private final AtomicInteger refCount = new AtomicInteger(0); + private final AtomicBoolean evicted = new AtomicBoolean(false); + private final AtomicBoolean closed = new AtomicBoolean(false); + + SectionReaderHolder(SectionReader sectionReader) { + this.sectionReader = sectionReader; + } + + SectionReader getSectionReader() { + return sectionReader; + } + + void retain() { + int refs = refCount.incrementAndGet(); + if (LOG.isTraceEnabled()) { + LOG.trace("Retained section reader {} (refCount={})", + Bytes.toStringBinary(sectionReader.tenantSectionId), refs); + } + } + + void release(boolean evictOnClose) { + int refs = refCount.decrementAndGet(); + if (refs < 0) { + LOG.warn("Section reader {} released too many times", + Bytes.toStringBinary(sectionReader.tenantSectionId)); + return; + } + if (refs == 0 && (evicted.get() || evictOnClose)) { + closeInternal(evictOnClose); + } + } + + void markEvicted(boolean evictOnClose) { + evicted.set(true); + if (refCount.get() == 0) { + closeInternal(evictOnClose); + } + } + + void forceClose(boolean evictOnClose) { + evicted.set(true); + closeInternal(evictOnClose); + } + + private void closeInternal(boolean evictOnClose) { + if (closed.compareAndSet(false, true)) { + try { + sectionReader.close(evictOnClose); + } catch (IOException e) { + LOG.warn("Failed to close section reader {}", + Bytes.toStringBinary(sectionReader.tenantSectionId), e); + } + } + } + + @Override + public String toString() { + return "SectionReaderHolder{" + "tenant=" + + Bytes.toStringBinary(sectionReader.tenantSectionId) + ", refCount=" + refCount.get() + + ", evicted=" + evicted.get() + ", closed=" + closed.get() + '}'; + } + } + + /** + * Lease handle giving callers access to a cached section reader while ensuring proper release. + */ + protected static final class SectionReaderLease implements AutoCloseable { + private final ImmutableBytesWritable cacheKey; + private final SectionReaderHolder holder; + private final SectionReader sectionReader; + private boolean closed; + + SectionReaderLease(ImmutableBytesWritable cacheKey, SectionReaderHolder holder) { + this.cacheKey = cacheKey; + this.holder = holder; + this.sectionReader = holder.getSectionReader(); + } + + public SectionReader getSectionReader() { + return sectionReader; + } + + public HFileReaderImpl getReader() throws IOException { + return sectionReader.getReader(); + } + + @Override + public void close() { + release(false); + } + + public void close(boolean evictOnClose) { + release(evictOnClose); + } + + private void release(boolean evictOnClose) { + if (closed) { + return; + } + closed = true; + holder.release(evictOnClose); + if (LOG.isTraceEnabled()) { + LOG.trace("Released lease for tenant {} (cacheKey={})", + Bytes.toStringBinary(sectionReader.tenantSectionId), cacheKey); + } + } + } + + /** + * Scanner implementation for multi-tenant HFiles. + *

+ * This scanner provides transparent access across multiple tenant sections by: + *

    + *
  • Extracting tenant information from seek keys
  • + *
  • Routing operations to the appropriate section reader
  • + *
  • Managing section transitions during sequential scans
  • + *
  • Optimizing performance through section prefetching
  • + *
+ */ + protected class MultiTenantScanner implements HFileScanner { + /** Configuration to use */ + protected final Configuration conf; + /** Whether to cache blocks */ + protected final boolean cacheBlocks; + /** Whether to use positional read */ + protected final boolean pread; + /** Whether this is for a compaction */ + protected final boolean isCompaction; + + /** Current tenant section ID */ + protected byte[] currentTenantSectionId; + /** Current scanner instance */ + protected HFileScanner currentScanner; + /** Current section reader lease */ + protected SectionReaderLease currentSectionLease; + /** Current section reader */ + protected SectionReader currentSectionReader; + /** Whether we have successfully seeked */ + protected boolean seeked = false; + + /** + * Constructor for MultiTenantScanner. + * @param conf Configuration to use + * @param cacheBlocks Whether to cache blocks + * @param pread Whether to use positional read + * @param isCompaction Whether this is for a compaction + */ + public MultiTenantScanner(Configuration conf, boolean cacheBlocks, boolean pread, + boolean isCompaction) { + this.conf = conf; + this.cacheBlocks = cacheBlocks; + this.pread = pread; + this.isCompaction = isCompaction; + } + + /** + * Switch to a new section reader, properly managing resource cleanup. + * @param newReader The new section reader to switch to + * @param sectionId The section ID for the new reader + * @throws IOException If an error occurs during the switch + */ + private void switchToSectionReader(SectionReaderLease newLease, byte[] sectionId) + throws IOException { + if (currentScanner != null) { + currentScanner.close(); + currentScanner = null; + } + if (currentSectionLease != null) { + currentSectionLease.close(); + currentSectionLease = null; + } + + currentSectionReader = null; + currentTenantSectionId = null; + + if (newLease != null) { + currentSectionLease = newLease; + currentSectionReader = newLease.getSectionReader(); + currentTenantSectionId = sectionId; + try { + currentScanner = currentSectionReader.getScanner(conf, cacheBlocks, pread, isCompaction); + } catch (IOException | RuntimeException e) { + currentSectionLease.close(); + currentSectionLease = null; + currentSectionReader = null; + currentTenantSectionId = null; + throw e; + } + } + } + + @Override + public boolean isSeeked() { + return seeked && currentScanner != null && currentScanner.isSeeked(); + } + + @Override + public boolean seekTo() throws IOException { + // Get the first section from the section index + if (!sectionIds.isEmpty()) { + // Get the first section ID from the list + byte[] firstSectionId = sectionIds.get(0).get(); + + if (switchToSection(firstSectionId)) { + boolean result = currentScanner.seekTo(); + seeked = result; + return result; + } else { + LOG.debug("No section reader available for first section {}", + Bytes.toStringBinary(firstSectionId)); + } + } + + // If we reach here, no sections were found or seeking failed + seeked = false; + return false; + } + + private boolean switchToSection(byte[] sectionId) throws IOException { + if ( + currentTenantSectionId != null && currentScanner != null && currentSectionLease != null + && Bytes.equals(currentTenantSectionId, sectionId) + ) { + return true; + } + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + return false; + } + switchToSectionReader(lease, sectionId); + return true; + } + + @Override + public int seekTo(ExtendedCell key) throws IOException { + // Handle empty or null keys by falling back to seekTo() without parameters + if (key == null || key.getRowLength() == 0) { + if (seekTo()) { + return 0; // Successfully seeked to first position + } else { + return -1; // No data found + } + } + + // Extract tenant section ID for the target key + byte[] targetSectionId = tenantExtractor.extractTenantSectionId(key); + + // Find insertion point for the target section in the sorted sectionIds list + int n = sectionIds.size(); + int insertionIndex = n; // default: after last + boolean exactSectionMatch = false; + for (int i = 0; i < n; i++) { + byte[] sid = sectionIds.get(i).get(); + int cmp = Bytes.compareTo(sid, targetSectionId); + if (cmp == 0) { + insertionIndex = i; + exactSectionMatch = true; + break; + } + if (cmp > 0) { + insertionIndex = i; + break; + } + } + + // If there is no exact section for this tenant prefix + if (!exactSectionMatch) { + if (insertionIndex == 0) { + // Key sorts before first section => before first key of entire file + seeked = false; + return -1; + } + return positionToPreviousSection(insertionIndex - 1); + } + + // Exact section exists. Seek within that section first. + byte[] matchedSectionId = sectionIds.get(insertionIndex).get(); + if (!switchToSection(matchedSectionId)) { + if (insertionIndex == 0) { + seeked = false; + return -1; + } + return positionToPreviousSection(insertionIndex - 1); + } + + int result = currentScanner.seekTo(key); + if (result == -1) { + // Key sorts before first key in this section. If this is the first section overall, + // then the key is before the first key in the entire file. + if (insertionIndex == 0) { + seeked = false; + return -1; + } + return positionToPreviousSection(insertionIndex - 1); + } + seeked = true; + return result; + } + + private int positionToPreviousSection(int startIndex) throws IOException { + for (int i = startIndex; i >= 0; i--) { + byte[] prevSectionId = sectionIds.get(i).get(); + if (!switchToSection(prevSectionId)) { + continue; + } + try { + Optional lastKeyOpt = currentSectionReader.getReader().getLastKey(); + if (lastKeyOpt.isPresent()) { + currentScanner.seekTo(lastKeyOpt.get()); + seeked = true; + return 1; + } + } catch (IOException e) { + LOG.warn("Failed to retrieve last key from section {}", + Bytes.toStringBinary(prevSectionId), e); + } + } + seeked = false; + return -1; + } + + @Override + public int reseekTo(ExtendedCell key) throws IOException { + if (!isSeeked()) { + return seekTo(key); + } + + // Extract tenant section ID + byte[] tenantSectionId = tenantExtractor.extractTenantSectionId(key); + + // If tenant section changed, we need to do a full seek + if (!Bytes.equals(tenantSectionId, currentTenantSectionId)) { + return seekTo(key); + } + + // Reuse existing scanner for same tenant section + int result = currentScanner.reseekTo(key); + if (result == -1) { + seeked = false; + } + return result; + } + + @Override + public boolean seekBefore(ExtendedCell key) throws IOException { + // Extract tenant section ID + byte[] tenantSectionId = tenantExtractor.extractTenantSectionId(key); + + // Get the scanner for this tenant section + if (!switchToSection(tenantSectionId)) { + seeked = false; + return false; + } + boolean result = currentScanner.seekBefore(key); + if (result) { + seeked = true; + } else { + seeked = false; + } + + return result; + } + + @Override + public ExtendedCell getCell() { + if (!isSeeked()) { + return null; + } + return currentScanner.getCell(); + } + + @Override + public ExtendedCell getKey() { + if (!isSeeked()) { + return null; + } + return currentScanner.getKey(); + } + + @Override + public java.nio.ByteBuffer getValue() { + if (!isSeeked()) { + return null; + } + return currentScanner.getValue(); + } + + @Override + public boolean next() throws IOException { + assertSeeked(); + + boolean hasNext = currentScanner.next(); + if (!hasNext) { + // Try to find the next tenant section + byte[] nextTenantSectionId = findNextTenantSectionId(currentTenantSectionId); + if (nextTenantSectionId == null) { + seeked = false; + return false; + } + + // Move to the next tenant section + if (!switchToSection(nextTenantSectionId)) { + seeked = false; + return false; + } + + // Prefetch the section after next if enabled + if (prefetchEnabled) { + prefetchNextSection(nextTenantSectionId); + } + + boolean result = currentScanner.seekTo(); + seeked = result; + return result; + } + + return true; + } + + /** + * Prefetch the next section after the given one for sequential access optimization. + * @param currentSectionId The current section ID + */ + private void prefetchNextSection(byte[] currentSectionId) { + try { + byte[] nextSectionId = findNextTenantSectionId(currentSectionId); + if (nextSectionId != null) { + // Trigger async load by creating the reader + SectionReaderLease lease = getSectionReader(nextSectionId); + if (lease != null) { + lease.close(); + } + } + } catch (Exception e) { + // Prefetch is best-effort, don't fail the operation + LOG.debug("Failed to prefetch next section", e); + } + } + + /** + * Find the next tenant section ID after the current one. + * @param currentSectionId The current section ID + * @return The next section ID, or null if none found + */ + private byte[] findNextTenantSectionId(byte[] currentSectionId) { + // Linear search to find current position and return next + for (int i = 0; i < sectionIds.size(); i++) { + if (Bytes.equals(sectionIds.get(i).get(), currentSectionId)) { + // Found current section, return next if it exists + if (i < sectionIds.size() - 1) { + return sectionIds.get(i + 1).get(); + } + break; + } + } + + return null; + } + + /** + * Assert that we have successfully seeked. + * @throws NotSeekedException if not seeked + */ + private void assertSeeked() { + if (!isSeeked()) { + throw new NotSeekedException(getPath()); + } + } + + @Override + public ExtendedCell getNextIndexedKey() { + if (!isSeeked()) { + return null; + } + return currentScanner.getNextIndexedKey(); + } + + @Override + public void close() { + if (currentScanner != null) { + currentScanner.close(); + currentScanner = null; + } + if (currentSectionLease != null) { + currentSectionLease.close(); + currentSectionLease = null; + } + currentSectionReader = null; + currentTenantSectionId = null; + seeked = false; + } + + @Override + public void shipped() throws IOException { + if (currentScanner != null) { + currentScanner.shipped(); + } + } + + @Override + public void recordBlockSize(java.util.function.IntConsumer blockSizeConsumer) { + if (currentScanner != null) { + currentScanner.recordBlockSize(blockSizeConsumer); + } + } + + @Override + public HFile.Reader getReader() { + return AbstractMultiTenantReader.this; + } + } + + /** + * Close all section readers and release resources. + * @throws IOException If an error occurs during close + */ + @Override + public void close() throws IOException { + close(false); + } + + /** + * Close underlying resources, with optional block eviction. + * @param evictOnClose Whether to evict blocks on close + * @throws IOException If an error occurs during close + */ + @Override + public void close(boolean evictOnClose) throws IOException { + sectionReaderCache.asMap().forEach((key, holder) -> { + if (holder != null) { + holder.forceClose(evictOnClose); + } + }); + sectionReaderCache.invalidateAll(); + sectionReaderCache.cleanUp(); + + // Close filesystem block reader streams + if (fsBlockReader != null) { + fsBlockReader.closeStreams(); + } + + // Unbuffer the main input stream wrapper + context.getInputStreamWrapper().unbuffer(); + } + + /** + * Get HFile version. + * @return The major version number + */ + @Override + public int getMajorVersion() { + return HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT; + } + + /** + * Build a section context with the appropriate offset translation wrapper. + *

+ * Creates a specialized reader context for a tenant section that handles: + *

    + *
  • Offset translation from section-relative to file-absolute positions
  • + *
  • Proper trailer positioning for HFile v3 section format
  • + *
  • Block boundary validation and alignment
  • + *
  • File size calculation for section boundaries
  • + *
+ * @param metadata The section metadata containing offset and size + * @param readerType The type of reader (PREAD or STREAM) + * @return A reader context for the section, or null if section is invalid + * @throws IOException If an error occurs building the context + */ + protected ReaderContext buildSectionContext(SectionMetadata metadata, + ReaderContext.ReaderType readerType) throws IOException { + // Create a special wrapper with offset translation capabilities + FSDataInputStreamWrapper parentWrapper = context.getInputStreamWrapper(); + MultiTenantFSDataInputStreamWrapper sectionWrapper = + new MultiTenantFSDataInputStreamWrapper(parentWrapper, metadata.getOffset()); + + // Calculate section size and validate minimum requirements + int sectionSize = metadata.getSize(); + int trailerSize = FixedFileTrailer.getTrailerSize(3); // HFile v3 sections use v3 format + + if (sectionSize < trailerSize) { + LOG.warn("Section size {} for offset {} is smaller than minimum trailer size {}", sectionSize, + metadata.getOffset(), trailerSize); + return null; + } + + // Build the reader context with proper file size calculation + // Use section size; wrapper handles offset translation + ReaderContext sectionContext = + ReaderContextBuilder.newBuilder(context).withInputStreamWrapper(sectionWrapper) + .withFilePath(context.getFilePath()).withReaderType(readerType) + .withFileSystem(context.getFileSystem()).withFileSize(sectionSize).build(); + + LOG.debug("Created section reader context for offset {}, size {}", metadata.getOffset(), + sectionSize); + return sectionContext; + } + + /** + * Get all tenant section IDs present in the file. + *

+ * Returns a defensive copy of all section IDs for external iteration without exposing internal + * data structures. + * @return An array of all tenant section IDs + */ + public byte[][] getAllTenantSectionIds() { + byte[][] allIds = new byte[sectionLocations.size()][]; + int i = 0; + for (ImmutableBytesWritable key : sectionLocations.keySet()) { + allIds[i++] = key.copyBytes(); + } + return allIds; + } + + /** + * For multi-tenant HFiles, get the first key from the first available section. This overrides the + * HFileReaderImpl implementation that requires dataBlockIndexReader. + * @return The first key if available + */ + @Override + public Optional getFirstKey() { + try { + // Get the first section and try to read its first key + for (ImmutableBytesWritable sectionKey : sectionLocations.keySet()) { + byte[] sectionId = sectionKey.get(); + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + continue; + } + try (lease) { + HFileReaderImpl reader = lease.getReader(); + Optional firstKey = reader.getFirstKey(); + if (firstKey.isPresent()) { + return firstKey; + } + } catch (IOException e) { + LOG.warn("Failed to get first key from section {}, trying next section", + Bytes.toString(sectionId), e); + // Continue to next section + } + } + + return Optional.empty(); + } catch (Exception e) { + LOG.error("Failed to get first key from multi-tenant HFile", e); + return Optional.empty(); + } + } + + /** + * For multi-tenant HFiles, get the last key from the last available section. This overrides the + * HFileReaderImpl implementation that requires dataBlockIndexReader. + * @return The last key if available + */ + @Override + public Optional getLastKey() { + try { + // Get the last section and try to read its last key + // Since LinkedHashMap maintains insertion order, iterate in reverse to get the last section + // first + List sectionKeys = new ArrayList<>(sectionLocations.keySet()); + for (int i = sectionKeys.size() - 1; i >= 0; i--) { + byte[] sectionId = sectionKeys.get(i).get(); + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + continue; + } + try (lease) { + HFileReaderImpl reader = lease.getReader(); + Optional lastKey = reader.getLastKey(); + if (lastKey.isPresent()) { + return lastKey; + } + } catch (IOException e) { + LOG.warn("Failed to get last key from section {}, trying previous section", + Bytes.toString(sectionId), e); + // Continue to previous section + } + } + + return Optional.empty(); + } catch (Exception e) { + LOG.error("Failed to get last key from multi-tenant HFile", e); + return Optional.empty(); + } + } + + /** + * For HFile v4 multi-tenant files, meta blocks don't exist at the file level. They exist within + * individual sections. This method is not supported. + * @param metaBlockName the name of the meta block to retrieve + * @param cacheBlock whether to cache the block + * @return always null for multi-tenant HFiles + * @throws IOException if an error occurs + */ + @Override + public HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException { + byte[] cachedSectionId = null; + if (metaBlockCacheEnabled && metaBlockSectionCache != null) { + ImmutableBytesWritable cached = metaBlockSectionCache.getIfPresent(metaBlockName); + if (cached != null) { + cachedSectionId = copySectionId(cached); + } + } + + if (cachedSectionId != null) { + HFileBlock cachedBlock = loadMetaBlockFromSection(cachedSectionId, metaBlockName, cacheBlock); + if (cachedBlock != null) { + return cachedBlock; + } + if (metaBlockCacheEnabled && metaBlockSectionCache != null) { + metaBlockSectionCache.invalidate(metaBlockName); + } + } + + if (sectionIds == null || sectionIds.isEmpty()) { + return null; + } + + for (ImmutableBytesWritable sectionId : sectionIds) { + byte[] candidateSectionId = copySectionId(sectionId); + if (cachedSectionId != null && Bytes.equals(candidateSectionId, cachedSectionId)) { + continue; + } + HFileBlock sectionBlock = + loadMetaBlockFromSection(candidateSectionId, metaBlockName, cacheBlock); + if (sectionBlock != null) { + if (metaBlockCacheEnabled && metaBlockSectionCache != null) { + metaBlockSectionCache.put(metaBlockName, new ImmutableBytesWritable(candidateSectionId)); + } + return sectionBlock; + } + } + + return null; + } + + /** + * For HFile v4 multi-tenant files, bloom filter metadata doesn't exist at the file level. It + * exists within individual sections. This method is not supported. + * @return always null for multi-tenant HFiles + * @throws IOException if an error occurs + */ + @Override + public DataInput getGeneralBloomFilterMetadata() throws IOException { + // HFile v4 multi-tenant files don't have file-level bloom filters + // Bloom filters exist within individual sections + LOG.debug( + "General bloom filter metadata not supported at file level for HFile v4 multi-tenant files"); + return null; + } + + /** + * For HFile v4 multi-tenant files, delete bloom filter metadata doesn't exist at the file level. + * It exists within individual sections. This method is not supported. + * @return always null for multi-tenant HFiles + * @throws IOException if an error occurs + */ + @Override + public DataInput getDeleteBloomFilterMetadata() throws IOException { + // HFile v4 multi-tenant files don't have file-level delete bloom filters + // Delete bloom filters exist within individual sections + LOG.debug( + "Delete bloom filter metadata not supported at file level for HFile v4 multi-tenant files"); + return null; + } + + private HFileBlockIndex.CellBasedKeyBlockIndexReader + loadDataBlockIndexFromSection(byte[] sectionId) { + SectionReaderLease lease; + try { + lease = getSectionReader(sectionId); + } catch (IOException e) { + LOG.debug("Failed to get section reader for section {}", Bytes.toStringBinary(sectionId), e); + return null; + } + if (lease == null) { + return null; + } + try (lease) { + HFileReaderImpl reader = lease.getReader(); + HFileBlockIndex.CellBasedKeyBlockIndexReader delegate = reader.getDataBlockIndexReader(); + if (delegate != null) { + setDataBlockIndexReader(delegate); + return delegate; + } + } catch (IOException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to load data block index from section {}", + Bytes.toStringBinary(sectionId), e); + } else { + LOG.warn("Failed to load data block index from section", e); + } + } + return null; + } + + private HFileBlock loadMetaBlockFromSection(byte[] sectionId, String metaBlockName, + boolean cacheBlock) throws IOException { + SectionReaderLease lease = getSectionReader(sectionId); + if (lease == null) { + return null; + } + try (lease) { + HFileReaderImpl reader = lease.getReader(); + return reader.getMetaBlock(metaBlockName, cacheBlock); + } + } + + private byte[] copySectionId(ImmutableBytesWritable sectionId) { + return Bytes.copy(sectionId.get(), sectionId.getOffset(), sectionId.getLength()); + } + + /** + * For HFile v4 multi-tenant files, index size is just the section index size. + * @return the heap size of the section index + */ + @Override + public long indexSize() { + if (sectionIndexReader != null) { + int numSections = sectionIndexReader.getNumSections(); + // Estimate: each section entry is approximately 64 bytes (prefix + offset + size) + return numSections * 64L; + } + return 0; + } + + /** + * Find a key at approximately the given position within a section. + * @param reader The section reader + * @param targetProgress The target position as a percentage (0.0 to 1.0) within the section + * @return A key near the target position, or empty if not found + */ + + /** + * Override mid-key calculation to find the middle key that respects tenant boundaries. For single + * tenant files, returns the midkey from the section. For multi-tenant files, finds the optimal + * tenant boundary that best balances the split. + * @return the middle key of the file + * @throws IOException if an error occurs + */ + @Override + public Optional midKey() throws IOException { + // Handle empty file case + if (sectionLocations.isEmpty()) { + LOG.debug("No sections in file, returning empty midkey"); + return Optional.empty(); + } + + // If there's only one section (single tenant), use that section's midkey + if (sectionLocations.size() == 1) { + byte[] sectionId = sectionLocations.keySet().iterator().next().get(); + try (SectionReaderLease lease = getSectionReader(sectionId)) { + if (lease == null) { + throw new IOException("Unable to create section reader for single tenant section: " + + Bytes.toStringBinary(sectionId)); + } + + HFileReaderImpl reader = lease.getReader(); + Optional midKey = reader.midKey(); + LOG.debug("Single tenant midkey: {}", midKey.orElse(null)); + return midKey; + } + } + + // For multiple tenants, find the optimal tenant boundary for splitting + // This ensures we never split within a tenant's data range + return findOptimalTenantBoundaryForSplit(); + } + + /** + * Find the optimal tenant boundary that best balances the region split. This method ensures that + * splits always occur at tenant boundaries, preserving tenant isolation and maintaining proper + * key ordering. + * @return the optimal boundary key for splitting + * @throws IOException if an error occurs + */ + private Optional findOptimalTenantBoundaryForSplit() throws IOException { + // Calculate total data volume and ideal split point + long totalFileSize = 0; + List tenantSections = new ArrayList<>(); + + for (Map.Entry entry : sectionLocations.entrySet()) { + SectionMetadata metadata = entry.getValue(); + totalFileSize += metadata.getSize(); + + // cumulative size up to this point + tenantSections + .add(new TenantSectionInfo(entry.getKey().get(), metadata.getSize(), totalFileSize)); + } + + if (totalFileSize == 0) { + LOG.debug("No data in file, returning empty midkey"); + return Optional.empty(); + } + + long idealSplitSize = totalFileSize / 2; + + // Find the tenant boundary that best approximates the ideal split + TenantSectionInfo bestBoundary = findBestTenantBoundary(tenantSections, idealSplitSize); + + if (bestBoundary == null) { + // Fallback: use the middle tenant if we can't find an optimal boundary + int middleTenantIndex = tenantSections.size() / 2; + bestBoundary = tenantSections.get(middleTenantIndex); + LOG.debug("Using middle tenant as fallback boundary: {}", + Bytes.toStringBinary(bestBoundary.tenantSectionId)); + } + + // Get the first key of the selected tenant section as the split point + // This ensures the split happens exactly at the tenant boundary + try (SectionReaderLease lease = getSectionReader(bestBoundary.tenantSectionId)) { + if (lease == null) { + throw new IOException("Unable to create section reader for boundary tenant: " + + Bytes.toStringBinary(bestBoundary.tenantSectionId)); + } + + HFileReaderImpl reader = lease.getReader(); + Optional firstKey = reader.getFirstKey(); + + if (firstKey.isPresent()) { + LOG.info("Selected tenant boundary midkey: {} (tenant: {}, split balance: {}/{})", + firstKey.get(), Bytes.toStringBinary(bestBoundary.tenantSectionId), + bestBoundary.cumulativeSize - bestBoundary.sectionSize, totalFileSize); + return firstKey; + } + + // If we can't get the first key, try the section's lastkey as fallback + Optional sectionLastKey = reader.getLastKey(); + if (sectionLastKey.isPresent()) { + LOG.warn( + "Using section last key as fallback (tenant boundary not available): {} (tenant: {})", + sectionLastKey.get(), Bytes.toStringBinary(bestBoundary.tenantSectionId)); + return sectionLastKey; + } + + throw new IOException("Unable to get any key from selected boundary tenant: " + + Bytes.toStringBinary(bestBoundary.tenantSectionId)); + } + } + + /** + * Find the tenant boundary that provides the most balanced split. This uses a heuristic to find + * the boundary that gets closest to a 50/50 split while maintaining tenant isolation. + * @param tenantSections List of tenant sections with cumulative sizes + * @param idealSplitSize The ideal size for the first region after split + * @return The best tenant boundary, or null if none suitable + */ + private TenantSectionInfo findBestTenantBoundary(List tenantSections, + long idealSplitSize) { + TenantSectionInfo bestBoundary = null; + long bestDeviation = Long.MAX_VALUE; + + // Evaluate each potential tenant boundary + for (int i = 1; i < tenantSections.size(); i++) { // Start from 1 to exclude first tenant + TenantSectionInfo boundary = tenantSections.get(i); + + // Calculate how balanced this split would be + long leftSideSize = boundary.cumulativeSize - boundary.sectionSize; // Size before this tenant + long deviation = Math.abs(leftSideSize - idealSplitSize); + + // Prefer boundaries that create more balanced splits + if (deviation < bestDeviation) { + bestDeviation = deviation; + bestBoundary = boundary; + } + + LOG.debug("Evaluating tenant boundary: {} (left: {}, deviation: {})", + Bytes.toStringBinary(boundary.tenantSectionId), leftSideSize, deviation); + } + + // Only use a boundary if it's reasonably balanced (within 30% of ideal) + if (bestBoundary != null) { + long leftSideSize = bestBoundary.cumulativeSize - bestBoundary.sectionSize; + double balanceRatio = Math.abs((double) leftSideSize / idealSplitSize - 1.0); + + if (balanceRatio > 0.3) { // More than 30% deviation + LOG.warn("Best tenant boundary has poor balance ratio: {:.1f}% (tenant: {})", + balanceRatio * 100, Bytes.toStringBinary(bestBoundary.tenantSectionId)); + // Still return it - tenant boundary is more important than perfect balance + } + } + + return bestBoundary; + } + + /** + * Helper class to track tenant section information for split analysis. + */ + private static class TenantSectionInfo { + final byte[] tenantSectionId; + final long sectionSize; + final long cumulativeSize; + + TenantSectionInfo(byte[] tenantSectionId, long sectionSize, long cumulativeSize) { + this.tenantSectionId = tenantSectionId; + this.sectionSize = sectionSize; + this.cumulativeSize = cumulativeSize; + } + } + + /** + * Override block reading to support tenant-aware block access. Routes block reads to the + * appropriate section based on offset. + * @param dataBlockOffset the offset of the block to read + * @param onDiskBlockSize the on-disk size of the block + * @param cacheBlock whether to cache the block + * @param pread whether to use positional read + * @param isCompaction whether this is for a compaction + * @param updateCacheMetrics whether to update cache metrics + * @param expectedBlockType the expected block type + * @param expectedDataBlockEncoding the expected data block encoding + * @return the read block + * @throws IOException if an error occurs reading the block + */ + @Override + public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, boolean cacheBlock, + boolean pread, boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, + DataBlockEncoding expectedDataBlockEncoding) throws IOException { + + try (SectionReaderLease lease = findSectionForOffset(dataBlockOffset)) { + if (lease == null) { + throw new IOException( + "No section found for offset: " + dataBlockOffset + ", path=" + getPath()); + } + + SectionReader targetSectionReader = lease.getSectionReader(); + HFileReaderImpl sectionReader = lease.getReader(); + + // Convert absolute offset to section-relative offset + long sectionRelativeOffset = dataBlockOffset - targetSectionReader.sectionBaseOffset; + + return sectionReader.readBlock(sectionRelativeOffset, onDiskBlockSize, cacheBlock, pread, + isCompaction, updateCacheMetrics, expectedBlockType, expectedDataBlockEncoding); + } catch (IOException e) { + LOG.error("Failed to read block at offset {} from section", dataBlockOffset, e); + throw e; + } + } + + /** + * Override block reading with section routing. + * @param dataBlockOffset the offset of the block to read + * @param onDiskBlockSize the on-disk size of the block + * @param cacheBlock whether to cache the block + * @param pread whether to use positional read + * @param isCompaction whether this is for a compaction + * @param updateCacheMetrics whether to update cache metrics + * @param expectedBlockType the expected block type + * @param expectedDataBlockEncoding the expected data block encoding + * @param cacheOnly whether to only read from cache + * @return the read block + * @throws IOException if an error occurs reading the block + */ + @Override + public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, boolean cacheBlock, + boolean pread, boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, + DataBlockEncoding expectedDataBlockEncoding, boolean cacheOnly) throws IOException { + + try (SectionReaderLease lease = findSectionForOffset(dataBlockOffset)) { + if (lease == null) { + throw new IOException( + "No section found for offset: " + dataBlockOffset + ", path=" + getPath()); + } + + SectionReader targetSectionReader = lease.getSectionReader(); + HFileReaderImpl sectionReader = lease.getReader(); + + // Convert absolute offset to section-relative offset + long sectionRelativeOffset = dataBlockOffset - targetSectionReader.sectionBaseOffset; + + return sectionReader.readBlock(sectionRelativeOffset, onDiskBlockSize, cacheBlock, pread, + isCompaction, updateCacheMetrics, expectedBlockType, expectedDataBlockEncoding, cacheOnly); + } catch (IOException e) { + LOG.error("Failed to read block at offset {} from section", dataBlockOffset, e); + throw e; + } + } + + /** + * Find the section reader that contains the given absolute file offset. + * @param absoluteOffset the absolute offset in the file + * @return the section reader containing this offset, or null if not found + */ + private SectionReaderLease findSectionForOffset(long absoluteOffset) throws IOException { + for (Map.Entry entry : sectionLocations.entrySet()) { + SectionMetadata metadata = entry.getValue(); + if ( + absoluteOffset >= metadata.getOffset() + && absoluteOffset < metadata.getOffset() + metadata.getSize() + ) { + return getSectionReader(entry.getKey().get()); + } + } + return null; + } + + /** + * For HFile v4 multi-tenant files, MVCC information is determined from file info only. + * @return true if file info indicates MVCC information is present + */ + @Override + public boolean hasMVCCInfo() { + // HFile v4 multi-tenant files determine MVCC info from file info only + return fileInfo.shouldIncludeMemStoreTS() && fileInfo.isDecodeMemstoreTS(); + } + + /** + * For HFile v4 multi-tenant files, entry count is determined from trailer only. + * @return the entry count from the trailer + */ + @Override + public long getEntries() { + // HFile v4 multi-tenant files get entry count from trailer only + if (trailer != null) { + return trailer.getEntryCount(); + } + return 0; + } + + /** + * Override unbuffer stream to handle main context. + */ + @Override + public void unbufferStream() { + // Unbuffer the main context + super.unbufferStream(); + + // Section readers are created on demand and managed by scanner + } + + /** + * For HFile v4 multi-tenant files, effective encoding is ignored. + * @param isCompaction whether this is for a compaction + * @return always NONE for multi-tenant HFiles + */ + @Override + public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) { + // HFile v4 multi-tenant files ignore effective encoding + LOG.debug("Effective encoding ignored for HFile v4 multi-tenant files"); + return DataBlockEncoding.NONE; + } + + /** + * Get section-specific statistics for monitoring and debugging. + * @return a map of section statistics + */ + public Map getSectionStatistics() { + Map stats = new HashMap<>(); + + stats.put("totalSections", sectionLocations.size()); + stats.put("tenantIndexLevels", tenantIndexLevels); + stats.put("tenantIndexMaxChunkSize", tenantIndexMaxChunkSize); + stats.put("prefetchEnabled", prefetchEnabled); + stats.put("cachedSectionReaders", sectionReaderCache.size()); + + // Section size distribution + List sectionSizes = new ArrayList<>(); + for (SectionMetadata metadata : sectionLocations.values()) { + sectionSizes.add(metadata.getSize()); + } + if (!sectionSizes.isEmpty()) { + stats.put("avgSectionSize", + sectionSizes.stream().mapToInt(Integer::intValue).average().orElse(0.0)); + stats.put("minSectionSize", + sectionSizes.stream().mapToInt(Integer::intValue).min().orElse(0)); + stats.put("maxSectionSize", + sectionSizes.stream().mapToInt(Integer::intValue).max().orElse(0)); + } + + return stats; + } + + /** + * Get metadata for a specific tenant section by section ID. + * @param tenantSectionId The tenant section ID to look up + * @return Detailed metadata about the section + */ + public Map getSectionInfo(byte[] tenantSectionId) { + Map info = new HashMap<>(); + + ImmutableBytesWritable key = new ImmutableBytesWritable(tenantSectionId); + SectionMetadata metadata = sectionLocations.get(key); + + if (metadata != null) { + info.put("exists", true); + info.put("offset", metadata.getOffset()); + info.put("size", metadata.getSize()); + } else { + info.put("exists", false); + } + + return info; + } + + /** + * Backward-compatibility shim for v3 expectations. + *

+ * Some existing code paths and unit tests (e.g. TestSeekTo) expect that a reader exposes a + * non-null data block index at the file level. For HFile v4 multi-tenant containers there is no + * global data index. When the container holds exactly one tenant section (the common case when + * multi-tenant writing is disabled), we can safely delegate to that section's v3 reader and + * expose its data block index to preserve v3 semantics. + */ + @Override + public HFileBlockIndex.CellBasedKeyBlockIndexReader getDataBlockIndexReader() { + HFileBlockIndex.CellBasedKeyBlockIndexReader existing = super.getDataBlockIndexReader(); + if (existing != null) { + return existing; + } + + byte[] hint = dataBlockIndexSectionHint.get(); + if (hint != null) { + HFileBlockIndex.CellBasedKeyBlockIndexReader delegate = loadDataBlockIndexFromSection(hint); + if (delegate != null) { + return delegate; + } + dataBlockIndexSectionHint.compareAndSet(hint, null); + } + + if (sectionIds == null || sectionIds.isEmpty()) { + return null; + } + + for (ImmutableBytesWritable sectionId : sectionIds) { + byte[] candidateSectionId = copySectionId(sectionId); + if (hint != null && Bytes.equals(candidateSectionId, hint)) { + continue; + } + HFileBlockIndex.CellBasedKeyBlockIndexReader delegate = + loadDataBlockIndexFromSection(candidateSectionId); + if (delegate != null) { + if (dataBlockIndexCacheEnabled) { + dataBlockIndexSectionHint.compareAndSet(null, candidateSectionId); + } + return delegate; + } + } + + return null; + } + + /** + * For HFile v4 multi-tenant files, data block encoding is ignored at file level. + * @return always NONE for multi-tenant HFiles + */ + @Override + public DataBlockEncoding getDataBlockEncoding() { + // HFile v4 multi-tenant files ignore data block encoding at file level + LOG.debug("Data block encoding ignored for HFile v4 multi-tenant files"); + return DataBlockEncoding.NONE; + } + + /** + * Check if prefetch is complete for this multi-tenant file. + * @return true if prefetching is complete for all sections + */ + @Override + public boolean prefetchComplete() { + // For multi-tenant files, prefetch is complete when section loading is done + // This is a simpler check than per-section prefetch status + return true; // Multi-tenant files handle prefetch at section level + } + + /** + * Check if prefetch has started for this multi-tenant file. + * @return true if prefetching has started + */ + @Override + public boolean prefetchStarted() { + // Multi-tenant files start prefetch immediately on open + return prefetchEnabled; + } + + /** + * Get file length from the context. + * @return the file length in bytes + */ + @Override + public long length() { + return context.getFileSize(); + } + + /** + * Check if file info is loaded (always true for multi-tenant readers). + * @return true as file info is always loaded during construction + */ + public boolean isFileInfoLoaded() { + return true; + } + + /** + * Override getHFileInfo to properly load FileInfo metadata for v4 files. + *

+ * Since initMetaAndIndex() is skipped for v4 files, we need to manually load the FileInfo block + * to expose the metadata written during file creation. + *

+ * This method ensures that the FileInfo block is loaded on-demand when HFilePrettyPrinter or + * other tools request the file metadata. + * @return The HFileInfo object with loaded metadata + */ + @Override + public HFileInfo getHFileInfo() { + // For v4 files, ensure FileInfo block is loaded on-demand + if (fileInfo.isEmpty()) { + try { + loadFileInfoBlock(); + } catch (IOException e) { + LOG.error("Failed to load FileInfo block for multi-tenant HFile", e); + // Continue with empty fileInfo rather than throwing exception + } + } + + return fileInfo; + } + + /** + * Manually load the FileInfo block for multi-tenant HFiles. + *

+ * This method replicates the FileInfo loading logic from HFileInfo.loadMetaInfo() but adapted for + * the multi-tenant file structure. + * @throws IOException if an error occurs loading the FileInfo block + */ + private void loadFileInfoBlock() throws IOException { + FixedFileTrailer trailer = getTrailer(); + + // Get the FileInfo block offset from the trailer + long fileInfoOffset = trailer.getFileInfoOffset(); + if (fileInfoOffset == 0) { + LOG.debug("No FileInfo block found in multi-tenant HFile"); + return; + } + + // Access the input stream through the context + FSDataInputStreamWrapper fsWrapper = context.getInputStreamWrapper(); + FSDataInputStream fsdis = fsWrapper.getStream(fsWrapper.shouldUseHBaseChecksum()); + long originalPosition = fsdis.getPos(); + + try { + LOG.debug("Loading FileInfo block from offset {}", fileInfoOffset); + + // Read the FileInfo block + HFileBlock fileInfoBlock = + getUncachedBlockReader().readBlockData(fileInfoOffset, -1, true, false, false); + HFileBlock blockToRead = null; + + // Validate this is a FileInfo block + if (fileInfoBlock.getBlockType() != BlockType.FILE_INFO) { + throw new IOException("Expected FILE_INFO block at offset " + fileInfoOffset + ", found " + + fileInfoBlock.getBlockType()); + } + + // Parse the FileInfo data using the HFileInfo.read() method + try { + blockToRead = fileInfoBlock.unpack(getFileContext(), getUncachedBlockReader()); + try (DataInputStream dis = new DataInputStream(blockToRead.getByteStream())) { + fileInfo.read(dis); + } + applyFileInfoMetadataToContext(); + } finally { + if (blockToRead != null) { + blockToRead.release(); + if (blockToRead != fileInfoBlock) { + fileInfoBlock.release(); + } + } else { + fileInfoBlock.release(); + } + } + + LOG.debug("Successfully loaded FileInfo with {} entries", fileInfo.size()); + } catch (IOException e) { + LOG.error("Failed to load FileInfo block from offset {}", fileInfoOffset, e); + throw e; + } finally { + // Restore original position + try { + fsdis.seek(originalPosition); + } catch (IOException e) { + LOG.warn("Failed to restore stream position", e); + } + } + } + + private void applyFileInfoMetadataToContext() { + HFileContext fileContext = getFileContext(); + + byte[] creationTimeBytes = fileInfo.get(HFileInfo.CREATE_TIME_TS); + if (creationTimeBytes != null) { + fileContext.setFileCreateTime(Bytes.toLong(creationTimeBytes)); + } + + byte[] maxTagsLenBytes = fileInfo.get(HFileInfo.MAX_TAGS_LEN); + boolean includesTags = maxTagsLenBytes != null; + fileContext.setIncludesTags(includesTags); + if (includesTags) { + byte[] tagsCompressedBytes = fileInfo.get(HFileInfo.TAGS_COMPRESSED); + boolean tagsCompressed = tagsCompressedBytes != null && Bytes.toBoolean(tagsCompressedBytes); + fileContext.setCompressTags(tagsCompressed); + } + + byte[] keyValueVersionBytes = fileInfo.get(HFileWriterImpl.KEY_VALUE_VERSION); + boolean includesMvcc = keyValueVersionBytes != null + && Bytes.toInt(keyValueVersionBytes) == HFileWriterImpl.KEY_VALUE_VER_WITH_MEMSTORE; + fileContext.setIncludesMvcc(includesMvcc); + } + + /** + * Enhanced toString with multi-tenant specific information. + * @return detailed string representation of this reader + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("MultiTenantReader{"); + sb.append("path=").append(getPath()); + sb.append(", majorVersion=").append(getMajorVersion()); + sb.append(", sections=").append(sectionLocations.size()); + sb.append(", tenantIndexLevels=").append(tenantIndexLevels); + sb.append(", fileSize=").append(length()); + + if (!sectionLocations.isEmpty()) { + try { + Optional firstKey = getFirstKey(); + Optional lastKey = getLastKey(); + if (firstKey.isPresent()) { + sb.append(", firstKey=").append(firstKey.get().toString()); + } + if (lastKey.isPresent()) { + sb.append(", lastKey=").append(lastKey.get().toString()); + } + } catch (Exception e) { + LOG.debug("Failed to get keys for toString", e); + } + } + + sb.append("}"); + return sb.toString(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/DefaultTenantExtractor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/DefaultTenantExtractor.java new file mode 100644 index 000000000000..37089ce9a2b3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/DefaultTenantExtractor.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Default implementation of TenantExtractor that extracts tenant information based on configurable + * prefix length at the beginning of row keys. + */ +@InterfaceAudience.Private +public class DefaultTenantExtractor implements TenantExtractor { + /** The length of the tenant prefix to extract */ + private final int prefixLength; + + /** + * Constructor for DefaultTenantExtractor. + * @param prefixLength the length of the tenant prefix to extract from row keys + */ + public DefaultTenantExtractor(int prefixLength) { + this.prefixLength = prefixLength; + } + + @Override + public byte[] extractTenantId(Cell cell) { + return extractPrefix(cell); + } + + @Override + public byte[] extractTenantSectionId(Cell cell) { + // Tenant section ID is same as tenant ID + return extractPrefix(cell); + } + + /** + * Extract tenant prefix from a cell. + * @param cell The cell to extract tenant information from + * @return The tenant prefix as a byte array + */ + private byte[] extractPrefix(Cell cell) { + if (prefixLength <= 0) { + return HConstants.EMPTY_BYTE_ARRAY; + } + + // Get row length and ensure it's sufficient + int rowLength = cell.getRowLength(); + if (rowLength < prefixLength) { + throw new IllegalArgumentException("Row key too short for configured prefix length. " + + "Row key length: " + rowLength + ", required: " + prefixLength); + } + + // Create and populate result array - always from start of row + byte[] prefix = new byte[prefixLength]; + System.arraycopy(cell.getRowArray(), cell.getRowOffset(), prefix, 0, prefixLength); + return prefix; + } + + /** + * Get the tenant prefix length. + * @return The configured tenant prefix length + */ + @Override + public int getPrefixLength() { + return prefixLength; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java index 74b560022a8b..ac4fba1efa55 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java @@ -130,6 +130,19 @@ public class FixedFileTrailer { */ private byte[] encryptionKey; + /** + * Flag indicating if this file is a multi-tenant HFile + */ + private boolean isMultiTenant = false; + + /** + * The tenant prefix length for multi-tenant HFiles + */ + private int tenantPrefixLength = 0; + + /** Offset of the multi-tenant section index root block */ + private long sectionIndexOffset = -1L; + /** * The {@link HFile} format major version. */ @@ -211,6 +224,14 @@ HFileProtos.FileTrailerProto toProtobuf() { if (encryptionKey != null) { builder.setEncryptionKey(UnsafeByteOperations.unsafeWrap(encryptionKey)); } + // Set multi-tenant fields for v4 files + if (isMultiTenant) { + builder.setMultiTenant(isMultiTenant); + builder.setTenantPrefixLength(tenantPrefixLength); + if (sectionIndexOffset >= 0) { + builder.setSectionIndexOffset(sectionIndexOffset); + } + } return builder.build(); } @@ -313,6 +334,15 @@ void deserializeFromPB(DataInputStream inputStream) throws IOException { if (trailerProto.hasEncryptionKey()) { encryptionKey = trailerProto.getEncryptionKey().toByteArray(); } + if (trailerProto.hasMultiTenant()) { + isMultiTenant = trailerProto.getMultiTenant(); + } + if (trailerProto.hasTenantPrefixLength()) { + tenantPrefixLength = trailerProto.getTenantPrefixLength(); + } + if (trailerProto.hasSectionIndexOffset()) { + sectionIndexOffset = trailerProto.getSectionIndexOffset(); + } } /** @@ -362,6 +392,13 @@ public String toString() { if (majorVersion >= 3) { append(sb, "encryptionKey=" + (encryptionKey != null ? "PRESENT" : "NONE")); } + if (majorVersion >= 4) { + append(sb, "isMultiTenant=" + isMultiTenant); + if (isMultiTenant) { + append(sb, "tenantPrefixLength=" + tenantPrefixLength); + append(sb, "sectionIndexOffset=" + sectionIndexOffset); + } + } append(sb, "majorVersion=" + majorVersion); append(sb, "minorVersion=" + minorVersion); @@ -453,6 +490,14 @@ public void setLoadOnOpenOffset(long loadOnOpenDataOffset) { this.loadOnOpenDataOffset = loadOnOpenDataOffset; } + public long getSectionIndexOffset() { + return sectionIndexOffset; + } + + public void setSectionIndexOffset(long sectionIndexOffset) { + this.sectionIndexOffset = sectionIndexOffset; + } + public int getDataIndexCount() { return dataIndexCount; } @@ -667,4 +712,20 @@ private static int extractMinorVersion(int serializedVersion) { static int materializeVersion(int majorVersion, int minorVersion) { return ((majorVersion & 0x00ffffff) | (minorVersion << 24)); } + + public boolean isMultiTenant() { + return isMultiTenant; + } + + public void setMultiTenant(boolean multiTenant) { + isMultiTenant = multiTenant; + } + + public int getTenantPrefixLength() { + return tenantPrefixLength; + } + + public void setTenantPrefixLength(int tenantPrefixLength) { + this.tenantPrefixLength = tenantPrefixLength; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index a99eac4085e4..98afa8862a5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -138,13 +138,16 @@ public final class HFile { /** * Maximum supported HFile format version */ - public static final int MAX_FORMAT_VERSION = 3; + public static final int MAX_FORMAT_VERSION = 4; /** * Minimum HFile format version with support for persisting cell tags */ public static final int MIN_FORMAT_VERSION_WITH_TAGS = 3; + /** Version for HFiles that support multi-tenant workloads */ + public static final int MIN_FORMAT_VERSION_WITH_MULTI_TENANT = 4; + /** Default compression name: none. */ public final static String DEFAULT_COMPRESSION = DEFAULT_COMPRESSION_ALGORITHM.getName(); @@ -354,6 +357,8 @@ public static final WriterFactory getWriterFactory(Configuration conf, CacheConf + "in hbase-site.xml)"); case 3: return new HFile.WriterFactory(conf, cacheConf); + case 4: + return new MultiTenantHFileWriter.WriterFactory(conf, cacheConf); default: throw new IllegalArgumentException( "Cannot create writer for HFile " + "format version " + version); @@ -499,19 +504,29 @@ HFileScanner getScanner(Configuration conf, boolean cacheBlocks, boolean pread, public static Reader createReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf, Configuration conf) throws IOException { try { + FixedFileTrailer trailer = fileInfo.getTrailer(); + int majorVersion = trailer.getMajorVersion(); + + // Handle HFile V4 (multi-tenant) separately for both stream and pread modes + if (majorVersion == MIN_FORMAT_VERSION_WITH_MULTI_TENANT) { + LOG.debug("Opening MultiTenant HFile v4"); + return MultiTenantReaderFactory.create(context, fileInfo, cacheConf, conf); + } + + // For non-multi-tenant files, continue with existing approach if (context.getReaderType() == ReaderType.STREAM) { // stream reader will share trailer with pread reader, see HFileStreamReader#copyFields return new HFileStreamReader(context, fileInfo, cacheConf, conf); } - FixedFileTrailer trailer = fileInfo.getTrailer(); - switch (trailer.getMajorVersion()) { + + switch (majorVersion) { case 2: LOG.debug("Opening HFile v2 with v3 reader"); // Fall through. FindBugs: SF_SWITCH_FALLTHROUGH case 3: return new HFilePreadReader(context, fileInfo, cacheConf, conf); default: - throw new IllegalArgumentException("Invalid HFile version " + trailer.getMajorVersion()); + throw new IllegalArgumentException("Invalid HFile version " + majorVersion); } } catch (Throwable t) { IOUtils.closeQuietly(context.getInputStreamWrapper(), @@ -560,7 +575,10 @@ public static Reader createReader(FileSystem fs, Path path, CacheConfig cacheCon .withPrimaryReplicaReader(primaryReplicaReader).withReaderType(ReaderType.PREAD).build(); HFileInfo fileInfo = new HFileInfo(context, conf); Reader reader = createReader(context, fileInfo, cacheConf, conf); - fileInfo.initMetaAndIndex(reader); + if (fileInfo.getTrailer().getMajorVersion() != HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT) { + // Only initialize meta and index for non-multi-tenant files + fileInfo.initMetaAndIndex(reader); + } return reader; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileInfo.java index 2386e8d82a56..029bf0bdab83 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileInfo.java @@ -372,25 +372,46 @@ public void initMetaAndIndex(HFile.Reader reader) throws IOException { ReaderContext context = reader.getContext(); try { HFileBlock.FSReader blockReader = reader.getUncachedBlockReader(); + long loadOnOpenOffset = trailer.getLoadOnOpenDataOffset(); + boolean isMultiTenantFile = + trailer.getMajorVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT + && trailer.getSectionIndexOffset() >= 0; + if (isMultiTenantFile) { + loadOnOpenOffset = trailer.getSectionIndexOffset(); + } + // Initialize an block iterator, and parse load-on-open blocks in the following. - blockIter = blockReader.blockRange(trailer.getLoadOnOpenDataOffset(), - context.getFileSize() - trailer.getTrailerSize()); - // Data index. We also read statistics about the block index written after - // the root level. - HFileBlock dataBlockRootIndex = blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX); - HFileBlock metaBlockIndex = blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX); - loadMetaInfo(blockIter, hfileContext); - - HFileIndexBlockEncoder indexBlockEncoder = - HFileIndexBlockEncoderImpl.createFromFileInfo(this); - this.dataIndexReader = new HFileBlockIndex.CellBasedKeyBlockIndexReaderV2( - trailer.createComparator(), trailer.getNumDataIndexLevels(), indexBlockEncoder); - dataIndexReader.readMultiLevelIndexRoot(dataBlockRootIndex, trailer.getDataIndexCount()); - reader.setDataBlockIndexReader(dataIndexReader); - // Meta index. - this.metaIndexReader = new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1); - metaIndexReader.readRootIndex(metaBlockIndex, trailer.getMetaIndexCount()); - reader.setMetaBlockIndexReader(metaIndexReader); + blockIter = + blockReader.blockRange(loadOnOpenOffset, context.getFileSize() - trailer.getTrailerSize()); + if (isMultiTenantFile) { + HFileBlock sectionIndexRoot = blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX); + try { + loadMetaInfo(blockIter, hfileContext); + } finally { + sectionIndexRoot.release(); + } + this.dataIndexReader = null; + this.metaIndexReader = null; + reader.setDataBlockIndexReader(null); + reader.setMetaBlockIndexReader(null); + } else { + // Data index. We also read statistics about the block index written after + // the root level. + HFileBlock dataBlockRootIndex = blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX); + HFileBlock metaBlockIndex = blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX); + loadMetaInfo(blockIter, hfileContext); + + HFileIndexBlockEncoder indexBlockEncoder = + HFileIndexBlockEncoderImpl.createFromFileInfo(this); + this.dataIndexReader = new HFileBlockIndex.CellBasedKeyBlockIndexReaderV2( + trailer.createComparator(), trailer.getNumDataIndexLevels(), indexBlockEncoder); + dataIndexReader.readMultiLevelIndexRoot(dataBlockRootIndex, trailer.getDataIndexCount()); + reader.setDataBlockIndexReader(dataIndexReader); + // Meta index. + this.metaIndexReader = new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1); + metaIndexReader.readRootIndex(metaBlockIndex, trailer.getMetaIndexCount()); + reader.setMetaBlockIndexReader(metaIndexReader); + } reader.setDataBlockEncoder(HFileDataBlockEncoderImpl.createFromFileInfo(this)); // Load-On-Open info @@ -469,14 +490,17 @@ private void loadMetaInfo(HFileBlock.BlockIterator blockIter, HFileContext hfile } /** - * File version check is a little sloppy. We read v3 files but can also read v2 files if their - * content has been pb'd; files written with 0.98. + * File version check is a little sloppy. We read v4 and v3 files but can also read v2 files if + * their content has been pb'd; files written with 0.98. */ private void checkFileVersion(Path path) { int majorVersion = trailer.getMajorVersion(); if (majorVersion == getMajorVersion()) { return; } + if (majorVersion == HFile.MIN_FORMAT_VERSION_WITH_TAGS) { + return; + } int minorVersion = trailer.getMinorVersion(); if (majorVersion == 2 && minorVersion >= MIN_V2_MINOR_VERSION_WITH_PB) { return; @@ -494,7 +518,7 @@ public void close() { } public int getMajorVersion() { - return 3; + return 4; } public void setTrailer(FixedFileTrailer trailer) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 9900aa63cab6..9bbcacd934b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -86,6 +86,37 @@ /** * Implements pretty-printing functionality for {@link HFile}s. + *

+ * This tool supports all HFile versions (v2, v3, and v4) with version-specific enhancements: + *

    + *
  • HFile v2: Basic file inspection, metadata, block headers, and key/value display
  • + *
  • HFile v3: All v2 features plus tag support and encryption metadata
  • + *
  • HFile v4: All v3 features plus multi-tenant support, tenant section display, and + * enhanced metadata for tenant isolation
  • + *
+ *

+ * Key improvements for HFile v4 multi-tenant support: + *

    + *
  • Version-aware block index handling (graceful fallback for v4)
  • + *
  • Enhanced block header display with tenant-aware error handling
  • + *
  • Tenant-specific information display with -t option
  • + *
  • Tenant boundary detection in key/value output
  • + *
  • V4-specific trailer field display (multi-tenant flags, tenant prefix length)
  • + *
  • Tenant isolation considerations (suppressed last key)
  • + *
+ *

+ * Usage examples: + * + *

+ * # Basic metadata for any HFile version
+ * hbase hfile -m -f /path/to/hfile
+ *
+ * # Key/value pairs with tenant information (v4 files)
+ * hbase hfile -p -v -t -f /path/to/v4/hfile
+ *
+ * # Block analysis (works across all versions)
+ * hbase hfile -b -h -f /path/to/hfile
+ * 
*/ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) @InterfaceStability.Evolving @@ -107,12 +138,17 @@ public class HFilePrettyPrinter extends Configured implements Tool { private boolean checkFamily; private boolean isSeekToRow = false; private boolean checkMobIntegrity = false; + private boolean printTenantInfo = false; private Map> mobFileLocations; private static final int FOUND_MOB_FILES_CACHE_CAPACITY = 50; private static final int MISSING_MOB_FILES_CACHE_CAPACITY = 20; private PrintStream out = System.out; private PrintStream err = System.err; + // Configurable block display limits + private int maxBlocksToShow; + private static final int DEFAULT_MAX_BLOCKS = 50; + /** * The row which the user wants to specify and print all the KeyValues for. */ @@ -150,6 +186,9 @@ private void init() { "Print detailed statistics, including counts by range"); options.addOption("i", "checkMobIntegrity", false, "Print all cells whose mob files are missing"); + options.addOption("t", "tenantinfo", false, + "Print tenant information for multi-tenant HFiles (v4+)"); + options.addOption("l", "blocklimit", true, "Maximum number of blocks to display (default: 50)"); OptionGroup files = new OptionGroup(); files.addOption(new Option("f", "file", true, @@ -184,6 +223,22 @@ public boolean parseOptions(String args[]) throws ParseException, IOException { checkRow = cmd.hasOption("k"); checkFamily = cmd.hasOption("a"); checkMobIntegrity = cmd.hasOption("i"); + printTenantInfo = cmd.hasOption("t"); + + if (cmd.hasOption("l")) { + try { + int limit = Integer.parseInt(cmd.getOptionValue("l")); + if (limit > 0) { + maxBlocksToShow = limit; + } else { + err.println("Invalid block limit: " + limit + ". Must be a positive number."); + System.exit(-1); + } + } catch (NumberFormatException e) { + err.println("Invalid block limit format: " + cmd.getOptionValue("l")); + System.exit(-1); + } + } if (cmd.hasOption("f")) { files.add(new Path(cmd.getOptionValue("f"))); @@ -306,6 +361,9 @@ public int processFile(Path file, boolean checkRootDir) throws IOException { HFile.Reader reader = HFile.createReader(fs, file, CacheConfig.DISABLED, true, getConf()); Map fileInfo = reader.getHFileInfo(); + FixedFileTrailer trailer = reader.getTrailer(); + int majorVersion = trailer.getMajorVersion(); + boolean isV4 = majorVersion == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT; KeyValueStatsCollector fileStats = null; @@ -327,31 +385,20 @@ public int processFile(Path file, boolean checkRootDir) throws IOException { // print meta data if (shouldPrintMeta) { - printMeta(reader, fileInfo); + printMeta(reader, fileInfo, isV4); + } + + // print tenant information for v4 files + if (printTenantInfo && isV4) { + printTenantInformation(reader); } if (printBlockIndex) { - out.println("Block Index:"); - out.println(reader.getDataBlockIndexReader()); + printBlockIndex(reader, isV4); } if (printBlockHeaders) { - out.println("Block Headers:"); - /* - * TODO: this same/similar block iteration logic is used in HFileBlock#blockRange and - * TestLazyDataBlockDecompression. Refactor? - */ - FSDataInputStreamWrapper fsdis = new FSDataInputStreamWrapper(fs, file); - long fileSize = fs.getFileStatus(file).getLen(); - FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis.getStream(false), fileSize); - long offset = trailer.getFirstDataBlockOffset(), max = trailer.getLastDataBlockOffset(); - HFileBlock block; - while (offset <= max) { - block = reader.readBlock(offset, -1, /* cacheBlock */ false, /* pread */ false, - /* isCompaction */ false, /* updateCacheMetrics */ false, null, null); - offset += block.getOnDiskSizeWithHeader(); - out.println(block); - } + printBlockHeaders(reader, file, fs, isV4); } if (printStats) { @@ -363,12 +410,44 @@ public int processFile(Path file, boolean checkRootDir) throws IOException { return 0; } + /** + * Get the effective block limit based on user configuration. + * @return the effective block limit to use + */ + private int getEffectiveBlockLimit() { + // If user specified a custom limit (> 0), use it + if (maxBlocksToShow > 0) { + return maxBlocksToShow; + } + // Otherwise use default + return DEFAULT_MAX_BLOCKS; + } + private void scanKeysValues(Path file, KeyValueStatsCollector fileStats, HFileScanner scanner, byte[] row) throws IOException { Cell pCell = null; FileSystem fs = FileSystem.get(getConf()); Set foundMobFiles = new LinkedHashSet<>(FOUND_MOB_FILES_CACHE_CAPACITY); Set missingMobFiles = new LinkedHashSet<>(MISSING_MOB_FILES_CACHE_CAPACITY); + + // Check if this is a v4 file for enhanced output + boolean isV4 = false; + String currentTenantId = null; + try { + HFile.Reader reader = scanner.getReader(); + if ( + reader != null + && reader.getTrailer().getMajorVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT + ) { + isV4 = true; + if (verbose) { + out.println("Scanning HFile v4 - tenant boundaries may be shown"); + } + } + } catch (Exception e) { + // Continue without tenant-specific processing + } + do { ExtendedCell cell = scanner.getCell(); if (row != null && row.length != 0) { @@ -379,6 +458,19 @@ private void scanKeysValues(Path file, KeyValueStatsCollector fileStats, HFileSc continue; } } + + // For multi-tenant v4 files, try to extract tenant information + if (isV4 && printKey) { + String extractedTenantId = extractTenantIdFromCell(cell, scanner.getReader()); + if (extractedTenantId != null && !extractedTenantId.equals(currentTenantId)) { + if (currentTenantId != null) { + out.println("--- End of tenant section: " + currentTenantId + " ---"); + } + currentTenantId = extractedTenantId; + out.println("--- Start of tenant section: " + currentTenantId + " ---"); + } + } + // collect stats if (printStats) { fileStats.collect(cell, printStatRanges); @@ -395,6 +487,10 @@ private void scanKeysValues(Path file, KeyValueStatsCollector fileStats, HFileSc out.print(String.format(" T[%d]: %s", i++, tag.toString())); } } + // Show tenant ID if available and verbose mode is on + if (isV4 && verbose && currentTenantId != null) { + out.print(" [Tenant: " + currentTenantId + "]"); + } out.println(); } // check if rows are in order @@ -443,6 +539,43 @@ private void scanKeysValues(Path file, KeyValueStatsCollector fileStats, HFileSc pCell = cell; ++count; } while (scanner.next()); + + // Close final tenant section if we were tracking it + if (isV4 && printKey && currentTenantId != null) { + out.println("--- End of tenant section: " + currentTenantId + " ---"); + } + } + + /** + * Enhanced tenant ID extraction that uses trailer information when available. + */ + private String extractTenantIdFromCell(ExtendedCell cell, HFile.Reader reader) { + try { + FixedFileTrailer trailer = reader.getTrailer(); + int tenantPrefixLength = 4; // fallback default + + // For v4 files, always try to get the actual tenant prefix length from trailer + if (trailer.getMajorVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT) { + tenantPrefixLength = trailer.getTenantPrefixLength(); + } + + byte[] rowKey = CellUtil.cloneRow(cell); + if (rowKey.length >= tenantPrefixLength) { + return Bytes.toStringBinary(rowKey, 0, tenantPrefixLength); + } else { + // Row key is shorter than expected tenant prefix + if (verbose && rowKey.length > 0) { + err.println("Warning: Row key length (" + rowKey.length + + ") is shorter than tenant prefix length (" + tenantPrefixLength + ")"); + } + return rowKey.length > 0 ? Bytes.toStringBinary(rowKey) : null; + } + } catch (Exception e) { + if (verbose) { + err.println("Warning: Error extracting tenant ID from cell: " + e.getMessage()); + } + } + return null; } /** @@ -508,10 +641,19 @@ private static String asSeparateLines(String keyValueStr) { return keyValueStr.replaceAll(", ([a-zA-Z]+=)", ",\n" + FOUR_SPACES + "$1"); } - private void printMeta(HFile.Reader reader, Map fileInfo) throws IOException { + private void printMeta(HFile.Reader reader, Map fileInfo, boolean isV4) + throws IOException { out.println("Block index size as per heapsize: " + reader.indexSize()); out.println(asSeparateLines(reader.toString())); - out.println("Trailer:\n " + asSeparateLines(reader.getTrailer().toString())); + + FixedFileTrailer trailer = reader.getTrailer(); + out.println("Trailer:\n " + asSeparateLines(trailer.toString())); + + // Print v4-specific trailer information if available + if (isV4) { + printV4SpecificTrailerInfo(trailer); + } + out.println("Fileinfo:"); for (Map.Entry e : fileInfo.entrySet()) { out.print(FOUR_SPACES + Bytes.toString(e.getKey()) + " = "); @@ -532,6 +674,11 @@ private void printMeta(HFile.Reader reader, Map fileInfo) throws || Bytes.equals(e.getKey(), HFileInfo.AVG_VALUE_LEN) || Bytes.equals(e.getKey(), HFileWriterImpl.KEY_VALUE_VERSION) || Bytes.equals(e.getKey(), HFileInfo.MAX_TAGS_LEN) + || Bytes.equals(e.getKey(), Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_SECTION_COUNT)) + || Bytes.equals(e.getKey(), + Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_TENANT_INDEX_LEVELS)) + || Bytes.equals(e.getKey(), + Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_TENANT_INDEX_MAX_CHUNK)) ) { out.println(Bytes.toInt(e.getValue())); } else if ( @@ -548,6 +695,13 @@ private void printMeta(HFile.Reader reader, Map fileInfo) throws } } + // For v4 files, also print section-level trailers and FileInfo + if (isV4 && reader instanceof AbstractMultiTenantReader) { + printSectionTrailers((AbstractMultiTenantReader) reader); + printSectionFileInfo((AbstractMultiTenantReader) reader); + } + + // Mid-key handling for different versions try { out.println("Mid-key: " + reader.midKey().map(CellUtil::getCellKeyAsString)); } catch (Exception e) { @@ -579,6 +733,585 @@ private void printMeta(HFile.Reader reader, Map fileInfo) throws } else { out.println(FOUR_SPACES + "Not present"); } + + // For v4 files, also print section-level bloom filter information + if (isV4 && reader instanceof AbstractMultiTenantReader) { + printSectionBloomFilters((AbstractMultiTenantReader) reader); + } + } + + /** + * Print trailer information for each section in a multi-tenant HFile v4. Each section is + * essentially an HFile v3 with its own trailer. + * @param mtReader the multi-tenant reader to get section information from + */ + private void printSectionTrailers(AbstractMultiTenantReader mtReader) { + byte[][] tenantSectionIds = mtReader.getAllTenantSectionIds(); + + if (tenantSectionIds == null || tenantSectionIds.length == 0) { + out.println("Section-level Trailers: No sections found"); + return; + } + + out.println("Section-level Trailers:"); + + for (int i = 0; i < tenantSectionIds.length; i++) { + byte[] sectionId = tenantSectionIds[i]; + out.println( + FOUR_SPACES + "--- Section " + i + ": " + Bytes.toStringBinary(sectionId) + " ---"); + + try ( + AbstractMultiTenantReader.SectionReaderLease lease = mtReader.getSectionReader(sectionId)) { + if (lease != null) { + HFileReaderImpl sectionHFileReader = lease.getReader(); + if (sectionHFileReader != null) { + FixedFileTrailer sectionTrailer = sectionHFileReader.getTrailer(); + if (sectionTrailer != null) { + out.println(FOUR_SPACES + FOUR_SPACES + "Section Trailer:"); + String trailerStr = sectionTrailer.toString(); + String[] lines = trailerStr.split("\n"); + for (String line : lines) { + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + line); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section trailer not available"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section reader not initialized"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Could not create section reader"); + } + } catch (IllegalArgumentException | IOException sectionException) { + out.println(FOUR_SPACES + FOUR_SPACES + "Error accessing section trailer: " + + sectionException.getMessage()); + } + + if (i < tenantSectionIds.length - 1) { + out.println(); // Add spacing between sections + } + } + } + + /** + * Print FileInfo for each section in a multi-tenant HFile v4. Each section is essentially an + * HFile v3 with its own FileInfo block. + * @param mtReader the multi-tenant reader to get section information from + */ + private void printSectionFileInfo(AbstractMultiTenantReader mtReader) { + byte[][] tenantSectionIds = mtReader.getAllTenantSectionIds(); + + if (tenantSectionIds == null || tenantSectionIds.length == 0) { + out.println("Section-level FileInfo: No sections found"); + return; + } + + out.println("Section-level FileInfo:"); + + for (int i = 0; i < tenantSectionIds.length; i++) { + byte[] sectionId = tenantSectionIds[i]; + out.println( + FOUR_SPACES + "--- Section " + i + ": " + Bytes.toStringBinary(sectionId) + " ---"); + + try ( + AbstractMultiTenantReader.SectionReaderLease lease = mtReader.getSectionReader(sectionId)) { + if (lease != null) { + HFileReaderImpl sectionHFileReader = lease.getReader(); + if (sectionHFileReader != null) { + Map sectionFileInfo = sectionHFileReader.getHFileInfo(); + if (sectionFileInfo != null && !sectionFileInfo.isEmpty()) { + out.println(FOUR_SPACES + FOUR_SPACES + "Section FileInfo:"); + for (Map.Entry e : sectionFileInfo.entrySet()) { + out.print( + FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + Bytes.toString(e.getKey()) + " = "); + if ( + Bytes.equals(e.getKey(), HStoreFile.MAX_SEQ_ID_KEY) + || Bytes.equals(e.getKey(), HStoreFile.DELETE_FAMILY_COUNT) + || Bytes.equals(e.getKey(), HStoreFile.EARLIEST_PUT_TS) + || Bytes.equals(e.getKey(), HFileWriterImpl.MAX_MEMSTORE_TS_KEY) + || Bytes.equals(e.getKey(), HFileInfo.CREATE_TIME_TS) + || Bytes.equals(e.getKey(), HStoreFile.BULKLOAD_TIME_KEY) + ) { + out.println(Bytes.toLong(e.getValue())); + } else if (Bytes.equals(e.getKey(), HStoreFile.TIMERANGE_KEY)) { + TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(e.getValue()); + out.println(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax()); + } else if ( + Bytes.equals(e.getKey(), HFileInfo.AVG_KEY_LEN) + || Bytes.equals(e.getKey(), HFileInfo.AVG_VALUE_LEN) + || Bytes.equals(e.getKey(), HFileWriterImpl.KEY_VALUE_VERSION) + || Bytes.equals(e.getKey(), HFileInfo.MAX_TAGS_LEN) + ) { + out.println(Bytes.toInt(e.getValue())); + } else if ( + Bytes.equals(e.getKey(), HStoreFile.MAJOR_COMPACTION_KEY) + || Bytes.equals(e.getKey(), HFileInfo.TAGS_COMPRESSED) + || Bytes.equals(e.getKey(), HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY) + || Bytes.equals(e.getKey(), HStoreFile.HISTORICAL_KEY) + ) { + out.println(Bytes.toBoolean(e.getValue())); + } else if (Bytes.equals(e.getKey(), HFileInfo.LASTKEY)) { + out.println(new KeyValue.KeyOnlyKeyValue(e.getValue()).toString()); + } else { + out.println(Bytes.toStringBinary(e.getValue())); + } + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section FileInfo not available or empty"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section reader not initialized"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Could not create section reader"); + } + } catch (IllegalArgumentException | IOException sectionException) { + out.println(FOUR_SPACES + FOUR_SPACES + "Error accessing section FileInfo: " + + sectionException.getMessage()); + } + + if (i < tenantSectionIds.length - 1) { + out.println(); // Add spacing between sections + } + } + } + + /** + * Print bloom filter information for each section in a multi-tenant HFile v4. + * @param mtReader the multi-tenant reader to get section information from + */ + private void printSectionBloomFilters(AbstractMultiTenantReader mtReader) { + byte[][] tenantSectionIds = mtReader.getAllTenantSectionIds(); + + if (tenantSectionIds == null || tenantSectionIds.length == 0) { + out.println("Section-level Bloom filters: No sections found"); + return; + } + + out.println("Section-level Bloom filters:"); + + for (int i = 0; i < tenantSectionIds.length; i++) { + byte[] sectionId = tenantSectionIds[i]; + out.println( + FOUR_SPACES + "--- Section " + i + ": " + Bytes.toStringBinary(sectionId) + " ---"); + + try ( + AbstractMultiTenantReader.SectionReaderLease lease = mtReader.getSectionReader(sectionId)) { + if (lease != null) { + HFileReaderImpl sectionHFileReader = lease.getReader(); + if (sectionHFileReader != null) { + + // Print general bloom filter for this section + DataInput bloomMeta = sectionHFileReader.getGeneralBloomFilterMetadata(); + BloomFilter bloomFilter = null; + if (bloomMeta != null) { + bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, sectionHFileReader); + } + + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + "General Bloom filter:"); + if (bloomFilter != null) { + String bloomDetails = bloomFilter.toString().replaceAll( + BloomFilterUtil.STATS_RECORD_SEP, "\n" + FOUR_SPACES + FOUR_SPACES + FOUR_SPACES); + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + bloomDetails); + } else { + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + "Not present"); + } + + // Print delete bloom filter for this section + bloomMeta = sectionHFileReader.getDeleteBloomFilterMetadata(); + bloomFilter = null; + if (bloomMeta != null) { + bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, sectionHFileReader); + } + + out.println(FOUR_SPACES + FOUR_SPACES + "Delete Family Bloom filter:"); + if (bloomFilter != null) { + String bloomDetails = bloomFilter.toString().replaceAll( + BloomFilterUtil.STATS_RECORD_SEP, "\n" + FOUR_SPACES + FOUR_SPACES + FOUR_SPACES); + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + bloomDetails); + } else { + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + "Not present"); + } + + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section reader not initialized"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Could not create section reader"); + } + } catch (IllegalArgumentException | IOException sectionException) { + out.println(FOUR_SPACES + FOUR_SPACES + "Error accessing section bloom filters: " + + sectionException.getMessage()); + } + + if (i < tenantSectionIds.length - 1) { + out.println(); // Add spacing between sections + } + } + } + + private void printV4SpecificTrailerInfo(FixedFileTrailer trailer) { + out.println("HFile v4 Specific Information:"); + try { + // Access v4-specific trailer fields directly (no reflection needed) + boolean isMultiTenant = trailer.isMultiTenant(); + out.println(FOUR_SPACES + "Multi-tenant enabled: " + isMultiTenant); + + if (isMultiTenant) { + int tenantPrefixLength = trailer.getTenantPrefixLength(); + out.println(FOUR_SPACES + "Tenant prefix length: " + tenantPrefixLength); + } + + } catch (Exception e) { + out.println( + FOUR_SPACES + "Unable to retrieve v4-specific trailer information: " + e.getMessage()); + } + } + + private void printTenantInformation(HFile.Reader reader) throws IOException { + out.println("Tenant Information:"); + + FixedFileTrailer trailer = reader.getTrailer(); + if (trailer.getMajorVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT) { + // Check if this is actually a multi-tenant file in the trailer + try { + // Access multi-tenant specific fields directly from trailer (no reflection needed) + boolean isMultiTenant = trailer.isMultiTenant(); + + if (isMultiTenant) { + out.println(FOUR_SPACES + "Multi-tenant: true"); + + int tenantPrefixLength = trailer.getTenantPrefixLength(); + out.println(FOUR_SPACES + "Tenant prefix length: " + tenantPrefixLength); + + // Try to access tenant section information if available + if (reader instanceof AbstractMultiTenantReader) { + AbstractMultiTenantReader mtReader = (AbstractMultiTenantReader) reader; + out.println(FOUR_SPACES + "Reader type: " + reader.getClass().getSimpleName()); + try { + byte[][] tenantSectionIds = mtReader.getAllTenantSectionIds(); + if (tenantSectionIds != null && tenantSectionIds.length > 0) { + out.println(FOUR_SPACES + "Number of tenant sections: " + tenantSectionIds.length); + for (int i = 0; i < Math.min(tenantSectionIds.length, 10); i++) { + out.println(FOUR_SPACES + "Tenant section " + i + ": " + + Bytes.toStringBinary(tenantSectionIds[i])); + } + if (tenantSectionIds.length > 10) { + out.println( + FOUR_SPACES + "... and " + (tenantSectionIds.length - 10) + " more sections"); + } + } + } catch (Exception e) { + out.println( + FOUR_SPACES + "Unable to retrieve tenant section information: " + e.getMessage()); + } + } + } else { + out.println(FOUR_SPACES + "Multi-tenant: false (HFile v4 format but single tenant)"); + } + } catch (Exception e) { + out.println(FOUR_SPACES + "Unable to retrieve multi-tenant information: " + e.getMessage()); + } + } else { + out.println( + FOUR_SPACES + "Not a multi-tenant HFile (version " + trailer.getMajorVersion() + ")"); + } + } + + private void printBlockIndex(HFile.Reader reader, boolean isV4) throws IOException { + out.println("Block Index:"); + + if (isV4) { + // For v4 files, show block index for each tenant section + if (reader instanceof AbstractMultiTenantReader) { + AbstractMultiTenantReader mtReader = (AbstractMultiTenantReader) reader; + byte[][] tenantSectionIds = mtReader.getAllTenantSectionIds(); + + if (tenantSectionIds != null && tenantSectionIds.length > 0) { + out.println( + FOUR_SPACES + "HFile v4 contains " + tenantSectionIds.length + " tenant sections:"); + + for (int i = 0; i < tenantSectionIds.length; i++) { + byte[] sectionId = tenantSectionIds[i]; + out.println(FOUR_SPACES + "--- Tenant Section " + i + ": " + + Bytes.toStringBinary(sectionId) + " ---"); + + try { + // Always show basic section information first + java.util.Map sectionInfo = mtReader.getSectionInfo(sectionId); + if (sectionInfo != null && (Boolean) sectionInfo.get("exists")) { + out.println( + FOUR_SPACES + FOUR_SPACES + "Section offset: " + sectionInfo.get("offset")); + out.println(FOUR_SPACES + FOUR_SPACES + "Section size: " + sectionInfo.get("size") + + " bytes"); + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section metadata not available"); + continue; + } + + // Get the actual block index from the section reader + try (AbstractMultiTenantReader.SectionReaderLease lease = + mtReader.getSectionReader(sectionId)) { + if (lease != null) { + HFileReaderImpl sectionHFileReader = lease.getReader(); + if (sectionHFileReader != null) { + HFileBlockIndex.CellBasedKeyBlockIndexReader indexReader = + sectionHFileReader.getDataBlockIndexReader(); + if (indexReader != null) { + out.println(FOUR_SPACES + FOUR_SPACES + "Block index details:"); + String indexDetails = indexReader.toString(); + // Indent the index details for better readability + String[] lines = indexDetails.split("\n"); + for (String line : lines) { + out.println(FOUR_SPACES + FOUR_SPACES + FOUR_SPACES + line); + } + } else { + out.println( + FOUR_SPACES + FOUR_SPACES + "Block index not available for this section"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section reader not initialized"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Could not create section reader"); + } + } catch (Exception sectionException) { + out.println(FOUR_SPACES + FOUR_SPACES + "Error accessing section reader: " + + sectionException.getMessage()); + } + + } catch (Exception e) { + out.println( + FOUR_SPACES + FOUR_SPACES + "Error reading section block index: " + e.getMessage()); + } + + if (i < tenantSectionIds.length - 1) { + out.println(); // Add spacing between sections + } + } + } else { + out.println(FOUR_SPACES + "No tenant sections found in HFile v4"); + } + } else { + out.println(FOUR_SPACES + "Reader is not a multi-tenant reader for v4 file"); + } + } else { + // For v2/v3 files, use standard approach + HFileBlockIndex.CellBasedKeyBlockIndexReader indexReader = reader.getDataBlockIndexReader(); + if (indexReader != null) { + out.println(indexReader); + } else { + out.println(FOUR_SPACES + "Block index not available"); + } + } + } + + private void printBlockHeaders(HFile.Reader reader, Path file, FileSystem fs, boolean isV4) + throws IOException { + out.println("Block Headers:"); + + if (isV4) { + // For v4 files, show block headers for each tenant section + if (reader instanceof AbstractMultiTenantReader) { + AbstractMultiTenantReader mtReader = (AbstractMultiTenantReader) reader; + byte[][] tenantSectionIds = mtReader.getAllTenantSectionIds(); + + if (tenantSectionIds != null && tenantSectionIds.length > 0) { + out.println( + FOUR_SPACES + "HFile v4 contains " + tenantSectionIds.length + " tenant sections:"); + + for (int i = 0; i < tenantSectionIds.length; i++) { + byte[] sectionId = tenantSectionIds[i]; + out.println(FOUR_SPACES + "--- Tenant Section " + i + ": " + + Bytes.toStringBinary(sectionId) + " ---"); + + try { + // Always show basic section information first + java.util.Map sectionInfo = mtReader.getSectionInfo(sectionId); + if (sectionInfo != null && (Boolean) sectionInfo.get("exists")) { + out.println( + FOUR_SPACES + FOUR_SPACES + "Section offset: " + sectionInfo.get("offset")); + out.println(FOUR_SPACES + FOUR_SPACES + "Section size: " + sectionInfo.get("size") + + " bytes"); + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section metadata not available"); + continue; + } + + // Get the actual block headers from the section reader + try (AbstractMultiTenantReader.SectionReaderLease lease = + mtReader.getSectionReader(sectionId)) { + if (lease != null) { + HFileReaderImpl sectionHFileReader = lease.getReader(); + if (sectionHFileReader != null) { + out.println(FOUR_SPACES + FOUR_SPACES + "Block headers:"); + // Create a section-specific path for block header reading + // Use the original file path since block reading handles section offsets + // internally + printSectionBlockHeaders(sectionHFileReader, file, fs, + FOUR_SPACES + FOUR_SPACES + FOUR_SPACES); + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Section reader not initialized"); + } + } else { + out.println(FOUR_SPACES + FOUR_SPACES + "Could not create section reader"); + } + } catch (Exception sectionException) { + out.println(FOUR_SPACES + FOUR_SPACES + "Error accessing section reader: " + + sectionException.getMessage()); + } + + } catch (Exception e) { + out.println(FOUR_SPACES + FOUR_SPACES + "Error reading section block headers: " + + e.getMessage()); + } + + if (i < tenantSectionIds.length - 1) { + out.println(); // Add spacing between sections + } + } + } else { + out.println(FOUR_SPACES + "No tenant sections found in HFile v4"); + } + } else { + out.println(FOUR_SPACES + "Reader is not a multi-tenant reader for v4 file"); + } + } else { + // For v2/v3 files, use standard approach + printStandardBlockHeaders(reader, file, fs); + } + } + + /** + * Print block headers using the standard approach for v2/v3 files. + */ + private void printStandardBlockHeaders(HFile.Reader reader, Path file, FileSystem fs) + throws IOException { + try { + FSDataInputStreamWrapper fsdis = new FSDataInputStreamWrapper(fs, file); + long fileSize = fs.getFileStatus(file).getLen(); + FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis.getStream(false), fileSize); + long offset = trailer.getFirstDataBlockOffset(); + long max = trailer.getLastDataBlockOffset(); + + if (offset > max || offset < 0 || max < 0) { + out.println(FOUR_SPACES + "Invalid block offset range: " + offset + " to " + max); + return; + } + + int blockCount = 0; + final int effectiveLimit = getEffectiveBlockLimit(); + + HFileBlock block; + while (offset <= max && blockCount < effectiveLimit) { + try { + block = reader.readBlock(offset, -1, /* cacheBlock */ false, /* pread */ false, + /* isCompaction */ false, /* updateCacheMetrics */ false, null, null); + + if (block == null) { + out.println(FOUR_SPACES + "Warning: null block at offset " + offset); + break; + } + + out.println(block); + offset += block.getOnDiskSizeWithHeader(); + blockCount++; + + } catch (Exception e) { + out.println( + FOUR_SPACES + "Error reading block at offset " + offset + ": " + e.getMessage()); + // For non-v4 files, try to continue with next logical offset + offset += 64; // Skip ahead and try again + if (offset > max) { + break; + } + } + } + + if (blockCount >= effectiveLimit) { + out.println(FOUR_SPACES + "... (truncated after " + effectiveLimit + " blocks)"); + } + + out.println(FOUR_SPACES + "Total blocks shown: " + blockCount); + + } catch (Exception e) { + out.println(FOUR_SPACES + "Unable to read block headers: " + e.getMessage()); + } + } + + /** + * Print block headers for a specific section reader with custom indentation. + * @param sectionReader the section reader to get block headers from + * @param file the original file path (for context) + * @param fs the file system + * @param indent the indentation string to use for output + * @throws IOException if an error occurs reading block headers + */ + private void printSectionBlockHeaders(HFileReaderImpl sectionReader, Path file, FileSystem fs, + String indent) throws IOException { + try { + FixedFileTrailer sectionTrailer = sectionReader.getTrailer(); + long firstDataBlockOffset = sectionTrailer.getFirstDataBlockOffset(); + long lastDataBlockOffset = sectionTrailer.getLastDataBlockOffset(); + + if (firstDataBlockOffset == -1 || lastDataBlockOffset == -1) { + out.println(indent + "No data blocks in this section"); + return; + } + + if ( + firstDataBlockOffset > lastDataBlockOffset || firstDataBlockOffset < 0 + || lastDataBlockOffset < 0 + ) { + out.println(indent + "Invalid block offset range: " + firstDataBlockOffset + " to " + + lastDataBlockOffset); + return; + } + + int blockCount = 0; + final int effectiveLimit = getEffectiveBlockLimit(); + long offset = firstDataBlockOffset; + + while (offset <= lastDataBlockOffset && blockCount < effectiveLimit) { + try { + HFileBlock block = + sectionReader.readBlock(offset, -1, /* cacheBlock */ false, /* pread */ false, + /* isCompaction */ false, /* updateCacheMetrics */ false, null, null); + + if (block == null) { + out.println(indent + "Warning: null block at offset " + offset); + break; + } + + // Print block header with proper indentation + String blockHeader = block.toString(); + String[] lines = blockHeader.split("\n"); + for (String line : lines) { + out.println(indent + line); + } + + offset += block.getOnDiskSizeWithHeader(); + blockCount++; + + } catch (Exception e) { + out.println(indent + "Error reading block at offset " + offset + ": " + e.getMessage()); + // Try to continue with next logical offset + offset += 64; // Skip ahead and try again + if (offset > lastDataBlockOffset) { + break; + } + } + } + + if (blockCount >= effectiveLimit) { + out.println(indent + "... (truncated after " + effectiveLimit + " blocks)"); + } + + out.println(indent + "Total blocks shown: " + blockCount); + + } catch (Exception e) { + out.println(indent + "Unable to read section block headers: " + e.getMessage()); + } } // Default reservoir is exponentially decaying, but we're doing a point-in-time analysis diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantBloomSupport.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantBloomSupport.java new file mode 100644 index 000000000000..e847955968b5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantBloomSupport.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.io.IOException; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.util.BloomFilter; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Internal contract that enables multi-tenant HFile readers to participate in Bloom filter + * decisions while staying transparent to existing StoreFileReader callers. + */ +@InterfaceAudience.Private +public interface MultiTenantBloomSupport { + + boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) throws IOException; + + boolean passesGeneralRowColBloomFilter(ExtendedCell cell) throws IOException; + + boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowLen) throws IOException; + + BloomType getGeneralBloomFilterType(); + + int getGeneralBloomPrefixLength() throws IOException; + + byte[] getLastBloomKey() throws IOException; + + long getDeleteFamilyBloomCount() throws IOException; + + BloomFilter getGeneralBloomFilterInstance() throws IOException; + + BloomFilter getDeleteFamilyBloomFilterInstance() throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileWriter.java new file mode 100644 index 000000000000..bac33247fa7f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileWriter.java @@ -0,0 +1,1558 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import static org.apache.hadoop.hbase.io.hfile.BlockCompressedSizePredicator.MAX_BLOCK_SIZE_UNCOMPRESSED; +import static org.apache.hadoop.hbase.io.hfile.HFileWriterImpl.KEY_VALUE_VERSION; +import static org.apache.hadoop.hbase.io.hfile.HFileWriterImpl.KEY_VALUE_VER_WITH_MEMSTORE; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.function.Supplier; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.io.Writable; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An HFile writer that supports multiple tenants by sectioning the data within a single file. + *

+ * This implementation takes advantage of the fact that HBase data is always written in sorted + * order, so once we move to a new tenant, we'll never go back to a previous one. + *

+ * Instead of creating separate physical files for each tenant, this writer creates a single HFile + * with internal sections that are indexed by tenant prefix. + *

+ * Key features: + *

    + *
  • Single HFile v4 format with multiple tenant sections
  • + *
  • Each section contains complete HFile v3 structure
  • + *
  • Section-level bloom filters for efficient tenant-specific queries
  • + *
  • Multi-level tenant indexing for fast section lookup
  • + *
  • Configurable tenant prefix extraction
  • + *
+ */ +@InterfaceAudience.Private +public class MultiTenantHFileWriter implements HFile.Writer { + private static final Logger LOG = LoggerFactory.getLogger(MultiTenantHFileWriter.class); + + /** Tenant identification configuration at cluster level */ + public static final String TENANT_PREFIX_LENGTH = "hbase.multi.tenant.prefix.length"; + + /** Tenant identification configuration at table level (higher precedence) */ + public static final String TABLE_TENANT_PREFIX_LENGTH = "TENANT_PREFIX_LENGTH"; + + /** Table-level property to enable/disable multi-tenant sectioning */ + public static final String TABLE_MULTI_TENANT_ENABLED = "MULTI_TENANT_HFILE"; + + /** FileInfo keys for multi-tenant HFile metadata */ + public static final String FILEINFO_SECTION_COUNT = "SECTION_COUNT"; + public static final String FILEINFO_TENANT_INDEX_LEVELS = "TENANT_INDEX_LEVELS"; + public static final String FILEINFO_TENANT_INDEX_MAX_CHUNK = "TENANT_INDEX_MAX_CHUNK"; + public static final String FILEINFO_TENANT_ID = "TENANT_ID"; + public static final String FILEINFO_TENANT_SECTION_ID = "TENANT_SECTION_ID"; + /** Empty prefix for default tenant */ + private static final byte[] DEFAULT_TENANT_PREFIX = new byte[0]; + + /** + * Class that manages tenant configuration with proper precedence: + *
    + *
  1. Table level settings have highest precedence
  2. + *
  3. Cluster level settings are used as fallback
  4. + *
  5. Default values are used if neither is specified
  6. + *
+ */ + // TenantConfiguration class removed - use TenantExtractorFactory instead + + /** Extractor for tenant information */ + private final TenantExtractor tenantExtractor; + /** Path for the HFile */ + private final Path path; + /** Configuration settings */ + private final Configuration conf; + /** Cache configuration */ + private final CacheConfig cacheConf; + /** HFile context */ + private final HFileContext fileContext; + /** Name used for logging/caching when path is not available */ + private final String streamName; + + /** Main file writer components - Output stream */ + private final FSDataOutputStream outputStream; + /** Whether this writer owns the underlying output stream */ + private final boolean closeOutputStream; + /** Block writer for HFile blocks */ + private HFileBlock.Writer blockWriter; + /** Section index writer for tenant indexing */ + private SectionIndexManager.Writer sectionIndexWriter; + + /** Section tracking - Current section writer */ + private SectionWriter currentSectionWriter; + /** Current tenant section ID */ + private byte[] currentTenantSectionId; + /** Start offset of current section */ + private long sectionStartOffset; + /** Number of sections written */ + private int sectionCount = 0; + + /** Stats for the entire file - Last cell written (internal tracking only) */ + private Cell lastCell = null; + /** Total number of entries */ + private long entryCount = 0; + /** Total key length across all entries */ + private long totalKeyLength = 0; + /** Total value length across all entries */ + private long totalValueLength = 0; + /** Length of the biggest cell */ + private long lenOfBiggestCell = 0; + /** Maximum memstore timestamp */ + private long maxMemstoreTS = 0; + /** Maximum tags length encountered */ + private int maxTagsLength = 0; + /** Aggregated time range across all sections */ + private final org.apache.hadoop.hbase.regionserver.TimeRangeTracker globalTimeRangeTracker = + org.apache.hadoop.hbase.regionserver.TimeRangeTracker + .create(org.apache.hadoop.hbase.regionserver.TimeRangeTracker.Type.NON_SYNC); + /** Aggregated custom tiering min timestamp */ + private long globalCustomMinTimestamp = + org.apache.hadoop.hbase.regionserver.TimeRangeTracker.INITIAL_MIN_TIMESTAMP; + /** Aggregated custom tiering max timestamp */ + private long globalCustomMaxTimestamp = + org.apache.hadoop.hbase.regionserver.TimeRangeTracker.INITIAL_MAX_TIMESTAMP; + /** Whether we have seen any custom time range metadata */ + private boolean globalCustomTimeRangePresent = false; + /** Supplier that exposes compaction-specific custom tiering time range tracking */ + private Supplier customTieringSupplier; + /** Earliest put timestamp across the file */ + private long globalEarliestPutTs = org.apache.hadoop.hbase.HConstants.LATEST_TIMESTAMP; + /** Bulk load timestamp for file info */ + private long bulkloadTime = 0; + /** Total uncompressed bytes */ + private long totalUncompressedBytes = 0; + /** Global maximum sequence id across sections */ + private long globalMaxSeqId = Long.MIN_VALUE; + + /** Absolute offset where each section's load-on-open data begins (max across sections) */ + private long maxSectionDataEndOffset = 0; + /** Absolute offset where the global section index root block starts */ + private long sectionIndexRootOffset = -1; + /** HFile v4 trailer */ + private FixedFileTrailer trailer; + /** File info for metadata */ + private HFileInfo fileInfo = new HFileInfo(); + /** Defaults to apply to each new section's FileInfo (e.g., compaction context) */ + private final HFileInfo sectionDefaultFileInfo = new HFileInfo(); + private static final byte[][] GLOBAL_FILE_INFO_KEYS = + new byte[][] { HStoreFile.BULKLOAD_TIME_KEY, HStoreFile.BULKLOAD_TASK_KEY, + HStoreFile.MAJOR_COMPACTION_KEY, HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY, + HStoreFile.COMPACTION_EVENT_KEY, HStoreFile.MAX_SEQ_ID_KEY, + HFileDataBlockEncoder.DATA_BLOCK_ENCODING, HFileIndexBlockEncoder.INDEX_BLOCK_ENCODING, + HFile.Writer.MAX_MEMSTORE_TS_KEY, HFileWriterImpl.KEY_VALUE_VERSION, + org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE }; + + /** Whether write verification is enabled */ + private boolean enableWriteVerification; + /** Configuration key for write verification */ + private static final String WRITE_VERIFICATION_ENABLED = + "hbase.multi.tenant.write.verification.enabled"; + /** Default write verification setting */ + private static final boolean DEFAULT_WRITE_VERIFICATION_ENABLED = false; + + /** Current bloom filter writer - one per section */ + private BloomFilterWriter currentBloomFilterWriter; + /** Whether bloom filter is enabled */ + private boolean bloomFilterEnabled; + /** Type of bloom filter to use */ + private BloomType bloomFilterType; + /** Per-section delete family bloom filter writer */ + private BloomFilterWriter currentDeleteFamilyBloomFilterWriter; + /** Per-section general bloom context for dedupe and LAST_BLOOM_KEY */ + private org.apache.hadoop.hbase.util.BloomContext currentGeneralBloomContext; + /** Per-section delete family bloom context */ + private org.apache.hadoop.hbase.util.BloomContext currentDeleteFamilyBloomContext; + /** Per-section time range tracker */ + private org.apache.hadoop.hbase.regionserver.TimeRangeTracker currentSectionTimeRangeTracker; + /** Per-section earliest put timestamp */ + private long currentSectionEarliestPutTs = org.apache.hadoop.hbase.HConstants.LATEST_TIMESTAMP; + /** Per-section delete family counter */ + private long currentSectionDeleteFamilyCnt = 0; + /** Per-section max sequence id */ + private long currentSectionMaxSeqId = 0; + /** Bloom param (e.g., rowprefix length) for the section */ + private byte[] currentGeneralBloomParam; + + /** + * Only these FileInfo keys are propagated as per-section defaults across tenant sections. This + * avoids unintentionally overriding section-local metadata. + */ + private static boolean isPropagatedDefaultKey(byte[] key) { + return Bytes.equals(key, org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY) + || Bytes.equals(key, org.apache.hadoop.hbase.regionserver.HStoreFile.COMPACTION_EVENT_KEY) + || Bytes.equals(key, org.apache.hadoop.hbase.regionserver.HStoreFile.HISTORICAL_KEY); + } + + /** + * Creates a multi-tenant HFile writer that writes sections to a single file. + * @param fs Filesystem to write to + * @param path Path for the HFile (final destination) + * @param conf Configuration settings + * @param cacheConf Cache configuration + * @param tenantExtractor Extractor for tenant information + * @param fileContext HFile context + * @param bloomType Type of bloom filter to use + * @throws IOException If an error occurs during initialization + */ + public MultiTenantHFileWriter(Path path, Configuration conf, CacheConfig cacheConf, + TenantExtractor tenantExtractor, HFileContext fileContext, BloomType bloomType, + FSDataOutputStream outputStream, boolean closeOutputStream) throws IOException { + this.path = path; + this.conf = conf; + this.cacheConf = cacheConf; + this.tenantExtractor = tenantExtractor; + this.fileContext = fileContext; + this.enableWriteVerification = + conf.getBoolean(WRITE_VERIFICATION_ENABLED, DEFAULT_WRITE_VERIFICATION_ENABLED); + + // Initialize bloom filter configuration using existing HBase properties + // This reuses the standard io.storefile.bloom.enabled property instead of creating + // a new multi-tenant specific property, ensuring consistency with existing HBase behavior + this.bloomFilterEnabled = BloomFilterFactory.isGeneralBloomEnabled(conf); + // Bloom filter type is passed from table properties, respecting column family configuration + this.bloomFilterType = bloomType; + + this.outputStream = Objects.requireNonNull(outputStream, "outputStream"); + this.closeOutputStream = closeOutputStream; + this.streamName = path != null ? path.toString() : this.outputStream.toString(); + + // Initialize bulk load timestamp for comprehensive file info + this.bulkloadTime = EnvironmentEdgeManager.currentTime(); + + // initialize blockWriter and sectionIndexWriter after creating stream + initialize(); + } + + /** + * Factory method to create a MultiTenantHFileWriter with configuration from both table and + * cluster levels. + *

+ * This method applies configuration precedence: + *

    + *
  1. Table-level properties have highest precedence
  2. + *
  3. Cluster-level configuration used as fallback
  4. + *
  5. Default values used if neither specified
  6. + *
+ * @param fs Filesystem to write to + * @param path Path for the HFile + * @param conf Configuration settings that include cluster-level tenant configuration + * @param cacheConf Cache configuration + * @param tableProperties Table properties that may include table-level tenant configuration + * @param fileContext HFile context + * @return A configured MultiTenantHFileWriter + * @throws IOException if writer creation fails + */ + public static MultiTenantHFileWriter create(FileSystem fs, Path path, Configuration conf, + CacheConfig cacheConf, Map tableProperties, HFileContext fileContext, + BloomType columnFamilyBloomType, BloomType defaultBloomType, FSDataOutputStream outputStream, + boolean closeOutputStream) throws IOException { + + FSDataOutputStream writerStream = outputStream; + boolean shouldCloseStream = closeOutputStream; + if (writerStream == null) { + Objects.requireNonNull(path, "path must be provided when outputStream is null"); + Objects.requireNonNull(fs, "filesystem must be provided when outputStream is null"); + writerStream = HFileWriterImpl.createOutputStream(conf, fs, path, null); + shouldCloseStream = true; + } + + // Create tenant extractor using factory - it will decide whether to use + // DefaultTenantExtractor or SingleTenantExtractor based on table properties + TenantExtractor tenantExtractor = + TenantExtractorFactory.createTenantExtractor(conf, tableProperties); + + // Determine bloom configuration with precedence: column family > table property > builder > + // default ROW + BloomType bloomType = columnFamilyBloomType; + if (tableProperties != null && tableProperties.containsKey("BLOOMFILTER")) { + try { + bloomType = BloomType.valueOf(tableProperties.get("BLOOMFILTER").toUpperCase()); + } catch (IllegalArgumentException e) { + LOG.warn("Invalid bloom filter type in table properties: {}, ignoring override", + tableProperties.get("BLOOMFILTER")); + } + } + if (bloomType == null) { + bloomType = defaultBloomType; + } + if (bloomType == null) { + bloomType = BloomType.ROW; + } + + LOG.info( + "Creating MultiTenantHFileWriter with tenant extractor: {}, bloom type: {} " + + "(cf override: {}, default: {}) and target {}", + tenantExtractor.getClass().getSimpleName(), bloomType, + columnFamilyBloomType != null ? columnFamilyBloomType : "", + defaultBloomType != null ? defaultBloomType : "", path != null ? path : writerStream); + + // HFile version 4 inherently implies multi-tenant + return new MultiTenantHFileWriter(path, conf, cacheConf, tenantExtractor, fileContext, + bloomType, writerStream, shouldCloseStream); + } + + /** + * Initialize the writer components including block writer and section index writer. + *

+ * Sets up multi-level tenant indexing with configurable chunk sizes and index parameters. + * @throws IOException if initialization fails + */ + private void initialize() throws IOException { + // Initialize the block writer + blockWriter = new HFileBlock.Writer(conf, NoOpDataBlockEncoder.INSTANCE, fileContext, + cacheConf.getByteBuffAllocator(), + conf.getInt(MAX_BLOCK_SIZE_UNCOMPRESSED, fileContext.getBlocksize() * 10)); + + // Initialize the section index using SectionIndexManager + boolean cacheIndexesOnWrite = cacheConf.shouldCacheIndexesOnWrite(); + String nameForCaching = null; + if (cacheIndexesOnWrite) { + nameForCaching = path != null ? path.getName() : streamName; + } + + sectionIndexWriter = new SectionIndexManager.Writer(blockWriter, + cacheIndexesOnWrite ? cacheConf : null, nameForCaching); + + // Configure multi-level tenant indexing based on configuration + int maxChunkSize = conf.getInt(SectionIndexManager.SECTION_INDEX_MAX_CHUNK_SIZE, + SectionIndexManager.DEFAULT_MAX_CHUNK_SIZE); + int minIndexNumEntries = conf.getInt(SectionIndexManager.SECTION_INDEX_MIN_NUM_ENTRIES, + SectionIndexManager.DEFAULT_MIN_INDEX_NUM_ENTRIES); + + sectionIndexWriter.setMaxChunkSize(maxChunkSize); + sectionIndexWriter.setMinIndexNumEntries(minIndexNumEntries); + + LOG.info("Initialized MultiTenantHFileWriter with multi-level section indexing for {} " + + "(maxChunkSize={}, minIndexNumEntries={})", streamName, maxChunkSize, minIndexNumEntries); + } + + @Override + public void append(ExtendedCell cell) throws IOException { + if (cell == null) { + throw new IOException("Cannot append null cell"); + } + + // Extract tenant section ID from the cell for section indexing + byte[] tenantSectionId = tenantExtractor.extractTenantSectionId(cell); + + // If this is the first cell or tenant section has changed, switch to new section + if (currentSectionWriter == null || !Arrays.equals(currentTenantSectionId, tenantSectionId)) { + if (currentSectionWriter != null) { + closeCurrentSection(); + } + // Extract tenant ID from the cell + byte[] tenantId = tenantExtractor.extractTenantId(cell); + createNewSection(tenantSectionId, tenantId); + } + + // Write the cell to the current section + currentSectionWriter.append(cell); + + // Update per-section metadata + // 1) General bloom (deduped by context) + if (bloomFilterEnabled && currentGeneralBloomContext != null) { + try { + currentGeneralBloomContext.writeBloom(cell); + } catch (IOException e) { + LOG.warn("Error adding cell to general bloom filter", e); + } + } + // 2) Delete family bloom and counter + if ( + org.apache.hadoop.hbase.PrivateCellUtil.isDeleteFamily(cell) + || org.apache.hadoop.hbase.PrivateCellUtil.isDeleteFamilyVersion(cell) + ) { + currentSectionDeleteFamilyCnt++; + if (currentDeleteFamilyBloomContext != null) { + try { + currentDeleteFamilyBloomContext.writeBloom(cell); + } catch (IOException e) { + LOG.warn("Error adding cell to delete family bloom filter", e); + } + } + } + // 3) Time range and earliest put ts + if (currentSectionTimeRangeTracker != null) { + currentSectionTimeRangeTracker.includeTimestamp(cell); + } + globalTimeRangeTracker.includeTimestamp(cell); + if (cell.getType() == Cell.Type.Put) { + long ts = cell.getTimestamp(); + currentSectionEarliestPutTs = Math.min(currentSectionEarliestPutTs, ts); + globalEarliestPutTs = Math.min(globalEarliestPutTs, ts); + } + // 4) Max seq id + if (cell.getSequenceId() > currentSectionMaxSeqId) { + currentSectionMaxSeqId = cell.getSequenceId(); + } + + // Track statistics for the entire file + lastCell = cell; + entryCount++; + totalKeyLength += PrivateCellUtil.estimatedSerializedSizeOfKey(cell); + totalValueLength += cell.getValueLength(); + + int cellSize = PrivateCellUtil.estimatedSerializedSizeOf(cell); + if (lenOfBiggestCell < cellSize) { + lenOfBiggestCell = cellSize; + } + + // Track maximum memstore timestamp across all cells + long cellMemstoreTS = cell.getSequenceId(); + if (cellMemstoreTS > maxMemstoreTS) { + maxMemstoreTS = cellMemstoreTS; + } + + int tagsLength = cell.getTagsLength(); + if (tagsLength > this.maxTagsLength) { + this.maxTagsLength = tagsLength; + } + } + + private void closeCurrentSection() throws IOException { + LOG.info("Closing section for tenant section ID: {}", + currentTenantSectionId == null ? "null" : Bytes.toStringBinary(currentTenantSectionId)); + + if (currentSectionWriter == null) { + LOG.warn("Attempted to close null section writer"); + return; + } + + try { + // Record the section start position + long sectionStartOffset = currentSectionWriter.getSectionStartOffset(); + + // Validate section has data + long entryCount = currentSectionWriter.getEntryCount(); + if (entryCount == 0) { + LOG.warn("Closing empty section for tenant: {}", + Bytes.toStringBinary(currentTenantSectionId)); + } + + // Add general bloom filter and metadata to the section if enabled + if (bloomFilterEnabled && currentBloomFilterWriter != null) { + long keyCount = currentBloomFilterWriter.getKeyCount(); + if (keyCount > 0) { + LOG.debug("Adding section-specific bloom filter with {} keys for section: {}", keyCount, + Bytes.toStringBinary(currentTenantSectionId)); + currentBloomFilterWriter.compactBloom(); + currentSectionWriter.addGeneralBloomFilter(currentBloomFilterWriter); + // Append bloom metadata similar to StoreFileWriter + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY, + Bytes.toBytes(bloomFilterType.toString())); + if (currentGeneralBloomParam != null) { + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_PARAM_KEY, + currentGeneralBloomParam); + } + // LAST_BLOOM_KEY + if (currentGeneralBloomContext != null) { + try { + currentGeneralBloomContext.addLastBloomKey(currentSectionWriter); + } catch (IOException e) { + LOG.warn("Failed to append LAST_BLOOM_KEY for section: {}", + Bytes.toStringBinary(currentTenantSectionId), e); + } + } + } else { + LOG.debug("No keys to add to general bloom filter for section: {}", + Bytes.toStringBinary(currentTenantSectionId)); + } + } + // Add delete family bloom filter and count + if (currentDeleteFamilyBloomFilterWriter != null) { + boolean hasDeleteFamilyBloom = currentDeleteFamilyBloomFilterWriter.getKeyCount() > 0; + if (hasDeleteFamilyBloom) { + currentDeleteFamilyBloomFilterWriter.compactBloom(); + currentSectionWriter.addDeleteFamilyBloomFilter(currentDeleteFamilyBloomFilterWriter); + } + } + // Always append delete family count + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT, + Bytes.toBytes(this.currentSectionDeleteFamilyCnt)); + + // Append per-section time range and earliest put ts + if (currentSectionTimeRangeTracker != null) { + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY, + org.apache.hadoop.hbase.regionserver.TimeRangeTracker + .toByteArray(currentSectionTimeRangeTracker)); + } + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS, + Bytes.toBytes(this.currentSectionEarliestPutTs)); + + // Append per-section MAX_SEQ_ID_KEY + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY, + Bytes.toBytes(this.currentSectionMaxSeqId)); + + // Finish writing the current section + currentSectionWriter.close(); + try { + outputStream.hsync(); // Ensure section data (incl. trailer) is synced to disk + } catch (UnsupportedOperationException uoe) { + outputStream.flush(); + } + + long sectionDataEnd = currentSectionWriter.getSectionDataEndOffset(); + if (sectionDataEnd >= 0) { + maxSectionDataEndOffset = Math.max(maxSectionDataEndOffset, sectionDataEnd); + } + + // Get current position to calculate section size + long sectionEndOffset = outputStream.getPos(); + long sectionSize = sectionEndOffset - sectionStartOffset; + + // Validate section size + if (sectionSize <= 0) { + throw new IOException("Invalid section size: " + sectionSize + " for tenant: " + + Bytes.toStringBinary(currentTenantSectionId)); + } + + // Validate section doesn't exceed max size (2GB limit for int) + if (sectionSize > Integer.MAX_VALUE) { + throw new IOException("Section size exceeds maximum: " + sectionSize + " for tenant: " + + Bytes.toStringBinary(currentTenantSectionId)); + } + + // Write verification if enabled + if (enableWriteVerification) { + verifySection(sectionStartOffset, sectionSize); + } + + // Record section in the index + sectionIndexWriter.addEntry(currentTenantSectionId, sectionStartOffset, (int) sectionSize); + + // Add to total uncompressed bytes + totalUncompressedBytes += currentSectionWriter.getTotalUncompressedBytes(); + + globalMaxSeqId = Math.max(globalMaxSeqId, currentSectionMaxSeqId); + + LOG.info("Section closed: start={}, size={}, entries={}", sectionStartOffset, sectionSize, + currentSectionWriter.getEntryCount()); + } catch (IOException e) { + LOG.error("Error closing section for tenant section ID: {}", + currentTenantSectionId == null ? "null" : Bytes.toStringBinary(currentTenantSectionId), e); + throw e; + } finally { + currentSectionWriter = null; + // Clear per-section trackers + currentBloomFilterWriter = null; + currentDeleteFamilyBloomFilterWriter = null; + currentGeneralBloomContext = null; + currentDeleteFamilyBloomContext = null; + currentSectionTimeRangeTracker = null; + currentSectionEarliestPutTs = org.apache.hadoop.hbase.HConstants.LATEST_TIMESTAMP; + currentSectionDeleteFamilyCnt = 0; + currentSectionMaxSeqId = 0; + currentGeneralBloomParam = null; + } + } + + /** + * Verify that the section was written correctly by checking basic structure. + *

+ * Performs basic validation of section size and structure without expensive I/O operations. + * @param sectionStartOffset Starting offset of the section in the file + * @param sectionSize Size of the section in bytes + * @throws IOException if verification fails or section structure is invalid + */ + private void verifySection(long sectionStartOffset, long sectionSize) throws IOException { + LOG.debug("Verifying section at offset {} with size {}", sectionStartOffset, sectionSize); + + // Basic verification: check that we can read the trailer + try { + // Seek to trailer position + int trailerSize = FixedFileTrailer.getTrailerSize(3); // v3 sections + long trailerOffset = sectionStartOffset + sectionSize - trailerSize; + + if (trailerOffset < sectionStartOffset) { + throw new IOException("Section too small to contain trailer: size=" + sectionSize); + } + + // Just verify the position is valid - actual trailer reading would require + // creating an input stream which is expensive + LOG.debug("Section verification passed: trailer would be at offset {}", trailerOffset); + } finally { + // Restore position + // Note: FSDataOutputStream doesn't support seek, so we can't actually verify + // Just log that verification was requested + LOG.debug("Write verification completed (limited check due to stream constraints)"); + } + } + + /** + * Create a new section for a tenant with its own writer and bloom filter. + *

+ * Each section is a complete HFile v3 structure within the larger v4 file. + * @param tenantSectionId The tenant section identifier for indexing + * @param tenantId The tenant identifier for metadata + * @throws IOException if section creation fails + */ + private void createNewSection(byte[] tenantSectionId, byte[] tenantId) throws IOException { + // Set the start offset for this section + sectionStartOffset = outputStream.getPos(); + + // Create a new virtual section writer + currentSectionWriter = new SectionWriter(conf, cacheConf, outputStream, fileContext, + tenantSectionId, tenantId, sectionStartOffset); + if (customTieringSupplier != null) { + currentSectionWriter.setTimeRangeTrackerForTiering(customTieringSupplier); + } + + // Initialize per-section trackers + this.currentSectionTimeRangeTracker = org.apache.hadoop.hbase.regionserver.TimeRangeTracker + .create(org.apache.hadoop.hbase.regionserver.TimeRangeTracker.Type.NON_SYNC); + this.currentSectionEarliestPutTs = org.apache.hadoop.hbase.HConstants.LATEST_TIMESTAMP; + this.currentSectionDeleteFamilyCnt = 0; + this.currentSectionMaxSeqId = 0; + + // Default per-section flags to ensure consistent presence across sections + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(false)); + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.HISTORICAL_KEY, Bytes.toBytes(false)); + try { + byte[] emptyEvent = org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil + .toCompactionEventTrackerBytes(java.util.Collections.emptySet()); + currentSectionWriter.appendFileInfo( + org.apache.hadoop.hbase.regionserver.HStoreFile.COMPACTION_EVENT_KEY, emptyEvent); + } catch (Exception e) { + LOG.debug("Unable to append default COMPACTION_EVENT_KEY for section: {}", + tenantSectionId == null ? "null" : Bytes.toStringBinary(tenantSectionId), e); + } + + // Apply only whitelisted section defaults (e.g., compaction context). Values here override + // above + for (java.util.Map.Entry e : sectionDefaultFileInfo.entrySet()) { + currentSectionWriter.appendFileInfo(e.getKey(), e.getValue()); + } + + // Create a new general bloom filter and contexts for this section if enabled + if (bloomFilterEnabled) { + currentBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf, + bloomFilterType, 0, currentSectionWriter); + if (currentBloomFilterWriter != null) { + // Create BloomContext matching type for dedupe and LAST_BLOOM_KEY + switch (bloomFilterType) { + case ROW: + currentGeneralBloomContext = new org.apache.hadoop.hbase.util.RowBloomContext( + currentBloomFilterWriter, fileContext.getCellComparator()); + break; + case ROWCOL: + currentGeneralBloomContext = new org.apache.hadoop.hbase.util.RowColBloomContext( + currentBloomFilterWriter, fileContext.getCellComparator()); + break; + case ROWPREFIX_FIXED_LENGTH: + currentGeneralBloomParam = org.apache.hadoop.hbase.util.BloomFilterUtil + .getBloomFilterParam(bloomFilterType, conf); + currentGeneralBloomContext = + new org.apache.hadoop.hbase.util.RowPrefixFixedLengthBloomContext( + currentBloomFilterWriter, fileContext.getCellComparator(), + org.apache.hadoop.hbase.util.Bytes.toInt(currentGeneralBloomParam)); + break; + default: + // Unsupported bloom type here should not happen as StoreFileWriter guards it + currentGeneralBloomContext = null; + break; + } + if (currentGeneralBloomParam == null) { + currentGeneralBloomParam = + org.apache.hadoop.hbase.util.BloomFilterUtil.getBloomFilterParam(bloomFilterType, conf); + } + } + // Initialize delete family bloom filter unless ROWCOL per StoreFileWriter semantics + if (bloomFilterType != BloomType.ROWCOL) { + currentDeleteFamilyBloomFilterWriter = + BloomFilterFactory.createDeleteBloomAtWrite(conf, cacheConf, 0, currentSectionWriter); + if (currentDeleteFamilyBloomFilterWriter != null) { + currentDeleteFamilyBloomContext = new org.apache.hadoop.hbase.util.RowBloomContext( + currentDeleteFamilyBloomFilterWriter, fileContext.getCellComparator()); + } + } + LOG.debug("Initialized bloom filters for tenant section ID: {}", + tenantSectionId == null ? "null" : Bytes.toStringBinary(tenantSectionId)); + } + + currentTenantSectionId = tenantSectionId; + sectionCount++; + + LOG.info("Created new section writer for tenant section ID: {}, tenant ID: {}, offset: {}", + tenantSectionId == null ? "null" : Bytes.toStringBinary(tenantSectionId), + tenantId == null ? "null" : Bytes.toStringBinary(tenantId), sectionStartOffset); + } + + @Override + public void close() throws IOException { + if (outputStream == null) { + return; + } + + // Ensure all sections are closed and resources flushed + if (currentSectionWriter != null) { + closeCurrentSection(); + currentSectionWriter = null; + } + + // HFile v4 structure: Section Index + File Info + Trailer + // (Each section contains complete HFile v3 with its own blocks) + // Note: v4 readers skip initMetaAndIndex, so no meta block index needed + + trailer = new FixedFileTrailer(getMajorVersion(), getMinorVersion()); + + // 1. Write Section Index Block (replaces data block index in v4) + // This is the core of HFile v4 - maps tenant prefixes to section locations + LOG.info("Writing section index with {} sections", sectionCount); + long rootIndexOffset = sectionIndexWriter.writeIndexBlocks(outputStream); + sectionIndexRootOffset = rootIndexOffset; + trailer.setSectionIndexOffset(sectionIndexRootOffset); + long loadOnOpenOffset = maxSectionDataEndOffset > 0 ? maxSectionDataEndOffset : rootIndexOffset; + if (loadOnOpenOffset > rootIndexOffset) { + // Clamp to ensure we never point past the actual section index start. + loadOnOpenOffset = rootIndexOffset; + } + trailer.setLoadOnOpenOffset(loadOnOpenOffset); + + // 2. Write File Info Block (minimal v4-specific metadata) + LOG.info("Writing v4 file info"); + finishFileInfo(); + writeFileInfo(trailer, blockWriter.startWriting(BlockType.FILE_INFO)); + blockWriter.writeHeaderAndData(outputStream); + totalUncompressedBytes += blockWriter.getUncompressedSizeWithHeader(); + + // 3. Write Trailer + finishClose(trailer); + + LOG.info( + "MultiTenantHFileWriter closed: target={}, sections={}, entries={}, " + + "totalUncompressedBytes={}", + streamName, sectionCount, entryCount, totalUncompressedBytes); + + blockWriter.release(); + } + + /** + * Write file info similar to HFileWriterImpl but adapted for multi-tenant structure. + * @param trailer The file trailer to update with file info offset + * @param out The output stream to write file info to + * @throws IOException if writing fails + */ + private void writeFileInfo(FixedFileTrailer trailer, DataOutputStream out) throws IOException { + trailer.setFileInfoOffset(outputStream.getPos()); + fileInfo.write(out); + } + + /** + * Finish the close for HFile v4 trailer. + *

+ * Sets v4-specific trailer fields including multi-tenant configuration and writes the final + * trailer to complete the file. + * @param trailer The trailer to finalize and write + * @throws IOException if trailer writing fails + */ + private void finishClose(FixedFileTrailer trailer) throws IOException { + // Set v4-specific trailer fields + trailer.setNumDataIndexLevels(sectionIndexWriter.getNumLevels()); + trailer.setUncompressedDataIndexSize(sectionIndexWriter.getTotalUncompressedSize()); + trailer.setDataIndexCount(sectionIndexWriter.getNumRootEntries()); + + // Set multi-tenant configuration in the trailer - MOST IMPORTANT PART + trailer.setMultiTenant(true); + trailer.setTenantPrefixLength(tenantExtractor.getPrefixLength()); + + // For v4 files, these indicate no global data blocks (data is in sections) + trailer.setFirstDataBlockOffset(-1); // UNSET indicates no global data blocks + trailer.setLastDataBlockOffset(-1); // UNSET indicates no global data blocks + + // Set other standard trailer fields + trailer.setComparatorClass(fileContext.getCellComparator().getClass()); + trailer.setMetaIndexCount(0); // No global meta blocks for multi-tenant files + trailer.setTotalUncompressedBytes(totalUncompressedBytes + trailer.getTrailerSize()); + trailer.setEntryCount(entryCount); + trailer.setCompressionCodec(fileContext.getCompression()); + + // Write trailer and close stream + long startTime = EnvironmentEdgeManager.currentTime(); + trailer.serialize(outputStream); + HFile.updateWriteLatency(EnvironmentEdgeManager.currentTime() - startTime); + + // Close the output stream - no file renaming needed since caller handles temporary files + try { + if (closeOutputStream) { + outputStream.close(); + LOG.info("Successfully closed MultiTenantHFileWriter: {}", streamName); + } else { + try { + outputStream.hflush(); + } catch (UnsupportedOperationException uoe) { + outputStream.flush(); + } + LOG.debug("Flushed MultiTenantHFileWriter output stream (caller retains ownership): {}", + streamName); + } + } catch (IOException e) { + LOG.error("Error finalizing MultiTenantHFileWriter for {}", streamName, e); + throw e; + } + } + + /** + * Finish file info preparation for multi-tenant HFile v4. + *

+ * Includes standard HFile metadata fields for compatibility with existing tooling, plus + * multi-tenant specific information. + * @throws IOException if file info preparation fails + */ + private void finishFileInfo() throws IOException { + // Don't store the last key in global file info for tenant isolation + // This is intentionally removed to ensure we don't track first/last keys globally + + // Average key length across all sections + int avgKeyLen = entryCount == 0 ? 0 : (int) (totalKeyLength / entryCount); + fileInfo.append(HFileInfo.AVG_KEY_LEN, Bytes.toBytes(avgKeyLen), false); + + // File creation timestamp + fileInfo.append(HFileInfo.CREATE_TIME_TS, Bytes.toBytes(fileContext.getFileCreateTime()), + false); + + // Average value length across all sections + int avgValueLength = entryCount == 0 ? 0 : (int) (totalValueLength / entryCount); + fileInfo.append(HFileInfo.AVG_VALUE_LEN, Bytes.toBytes(avgValueLength), false); + + // Biggest cell info (key removed for tenant isolation) + // Only store length which doesn't expose key information + if (lenOfBiggestCell > 0) { + fileInfo.append(HFileInfo.LEN_OF_BIGGEST_CELL, Bytes.toBytes(lenOfBiggestCell), false); + } + + // Bulk load timestamp - when this file was created/written + fileInfo.append(HStoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime), false); + + // Memstore and version metadata + if (fileContext.isIncludesMvcc()) { + fileInfo.append(MAX_MEMSTORE_TS_KEY, Bytes.toBytes(maxMemstoreTS), false); + fileInfo.append(KEY_VALUE_VERSION, Bytes.toBytes(KEY_VALUE_VER_WITH_MEMSTORE), false); + } + + // Tags metadata + if (fileContext.isIncludesTags()) { + fileInfo.append(HFileInfo.MAX_TAGS_LEN, Bytes.toBytes(maxTagsLength), false); + boolean tagsCompressed = (fileContext.getDataBlockEncoding() != DataBlockEncoding.NONE) + && fileContext.isCompressTags(); + fileInfo.append(HFileInfo.TAGS_COMPRESSED, Bytes.toBytes(tagsCompressed), false); + } + + // === MULTI-TENANT SPECIFIC METADATA (v4 enhancements) === + + // Section and tenant information + fileInfo.append(Bytes.toBytes(FILEINFO_SECTION_COUNT), Bytes.toBytes(sectionCount), false); + + // Tenant index structure information + int tenantIndexLevels = sectionIndexWriter.getNumLevels(); + fileInfo.append(Bytes.toBytes(FILEINFO_TENANT_INDEX_LEVELS), Bytes.toBytes(tenantIndexLevels), + false); + + // Store the configured max chunk size for tenant index + int maxChunkSize = conf.getInt(SectionIndexManager.SECTION_INDEX_MAX_CHUNK_SIZE, + SectionIndexManager.DEFAULT_MAX_CHUNK_SIZE); + fileInfo.append(Bytes.toBytes(FILEINFO_TENANT_INDEX_MAX_CHUNK), Bytes.toBytes(maxChunkSize), + false); + + // Standard compatibility metadata expected by existing tooling + if ( + globalTimeRangeTracker.getMax() + != org.apache.hadoop.hbase.regionserver.TimeRangeTracker.INITIAL_MAX_TIMESTAMP + ) { + fileInfo.append(org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY, + org.apache.hadoop.hbase.regionserver.TimeRangeTracker.toByteArray(globalTimeRangeTracker), + false); + } + if (globalEarliestPutTs != org.apache.hadoop.hbase.HConstants.LATEST_TIMESTAMP) { + fileInfo.append(org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS, + Bytes.toBytes(globalEarliestPutTs), false); + } + if (globalCustomTimeRangePresent) { + org.apache.hadoop.hbase.regionserver.TimeRangeTracker customTracker = + org.apache.hadoop.hbase.regionserver.TimeRangeTracker.create( + org.apache.hadoop.hbase.regionserver.TimeRangeTracker.Type.NON_SYNC, + globalCustomMinTimestamp, globalCustomMaxTimestamp); + fileInfo.append( + org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE, + org.apache.hadoop.hbase.regionserver.TimeRangeTracker.toByteArray(customTracker), false); + } + + if (globalMaxSeqId != Long.MIN_VALUE) { + fileInfo.put(HStoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(globalMaxSeqId)); + } + if (fileContext.isIncludesMvcc()) { + fileInfo.put(HFile.Writer.MAX_MEMSTORE_TS_KEY, Bytes.toBytes(maxMemstoreTS)); + } + } + + @Override + public void appendFileInfo(byte[] key, byte[] value) throws IOException { + if (shouldStoreInGlobalFileInfo(key) && fileInfo.get(key) == null) { + fileInfo.append(key, value, true); + } + // Propagate only known-safe defaults across sections + if (isPropagatedDefaultKey(key)) { + sectionDefaultFileInfo.append(key, value, true); + } + // If a section is active, also apply immediately + if (currentSectionWriter != null) { + currentSectionWriter.appendFileInfo(key, value); + } + } + + private boolean shouldStoreInGlobalFileInfo(byte[] key) { + for (byte[] allowed : GLOBAL_FILE_INFO_KEYS) { + if (Bytes.equals(allowed, key)) { + return true; + } + } + return false; + } + + @Override + public void appendMetaBlock(String metaBlockName, Writable content) { + try { + if (currentSectionWriter == null) { + if (sectionCount == 0) { + LOG.debug("No section available when appending meta block {}; creating default section", + metaBlockName); + createNewSection(DEFAULT_TENANT_PREFIX, DEFAULT_TENANT_PREFIX); + } else { + throw new IllegalStateException( + "Active section expected when appending meta block " + metaBlockName); + } + } + currentSectionWriter.appendMetaBlock(metaBlockName, content); + } catch (IOException e) { + throw new UncheckedIOException("Failed to append meta block " + metaBlockName, e); + } + } + + @Override + public void appendTrackedTimestampsToMetadata() throws IOException { + if (currentSectionWriter != null) { + currentSectionWriter.appendTrackedTimestampsToMetadata(); + } + } + + @Override + public void appendCustomCellTimestampsToMetadata( + org.apache.hadoop.hbase.regionserver.TimeRangeTracker timeRangeTracker) throws IOException { + if (timeRangeTracker != null) { + long max = timeRangeTracker.getMax(); + if (max != org.apache.hadoop.hbase.regionserver.TimeRangeTracker.INITIAL_MAX_TIMESTAMP) { + long min = timeRangeTracker.getMin(); + long effectiveMin = + min != org.apache.hadoop.hbase.regionserver.TimeRangeTracker.INITIAL_MIN_TIMESTAMP + ? min + : max; + globalCustomMinTimestamp = Math.min(globalCustomMinTimestamp, effectiveMin); + globalCustomMaxTimestamp = Math.max(globalCustomMaxTimestamp, max); + globalCustomTimeRangePresent = true; + } + } + if (currentSectionWriter != null) { + currentSectionWriter.appendCustomCellTimestampsToMetadata(timeRangeTracker); + } + } + + @Override + public void addInlineBlockWriter(InlineBlockWriter ibw) { + if (currentSectionWriter != null) { + currentSectionWriter.addInlineBlockWriter(ibw); + } + } + + @Override + public void addGeneralBloomFilter(BloomFilterWriter bfw) { + // For multi-tenant files, bloom filters are only added at section level + // We create and add a bloom filter for each section separately + // This method is called externally but we ignore it since we handle bloom filters internally + if (bfw == null || bfw.getKeyCount() <= 0) { + LOG.debug("Ignoring empty or null general bloom filter at global level"); + return; + } + + LOG.debug( + "Ignoring external bloom filter with {} keys - using per-section bloom filters instead", + bfw.getKeyCount()); + } + + @Override + public void addDeleteFamilyBloomFilter(BloomFilterWriter bfw) throws IOException { + // For multi-tenant files, bloom filters are only added at section level + // This prevents creating bloom filters at the global level + if (bfw == null || bfw.getKeyCount() <= 0) { + LOG.debug("Ignoring empty or null delete family bloom filter at global level"); + return; + } + + // Only add to current section if one exists + if (currentSectionWriter != null) { + LOG.debug("Delegating delete family bloom filter with {} keys to current section", + bfw.getKeyCount()); + // Ensure it's properly prepared for writing + bfw.compactBloom(); + currentSectionWriter.addDeleteFamilyBloomFilter(bfw); + } else { + LOG.warn("Attempted to add delete family bloom filter with {} keys but no section is active", + bfw.getKeyCount()); + } + } + + @Override + public void beforeShipped() throws IOException { + if (currentSectionWriter != null) { + currentSectionWriter.beforeShipped(); + } + + // Clone cells for thread safety if necessary + if (this.lastCell != null) { + this.lastCell = KeyValueUtil.toNewKeyCell((ExtendedCell) this.lastCell); + } + } + + @Override + public Path getPath() { + return path; + } + + /** + * Registers a supplier that exposes the custom tiering time range tracker so SectionWriter + * instances can share it with the core {@link HFileWriterImpl} logic (e.g., block caching). + */ + public void setCustomTieringTimeRangeSupplier( + Supplier supplier) { + this.customTieringSupplier = supplier; + if (currentSectionWriter != null) { + currentSectionWriter.setTimeRangeTrackerForTiering(supplier); + } + } + + @Override + public HFileContext getFileContext() { + return fileContext; + } + + public long getEntryCount() { + return entryCount; + } + + public Cell getLastCell() { + return lastCell; // Keep API, but note this won't be used in global structures + } + + /** + * The multi-tenant HFile writer always returns version 4, which is the first version to support + * multi-tenant HFiles. + * @return The major version for multi-tenant HFiles (4) + */ + protected int getMajorVersion() { + return HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT; + } + + /** + * The minor version of HFile format. + */ + protected int getMinorVersion() { + return 0; + } + + /** + * Get the current number of tenant sections. + * @return The section count + */ + public int getSectionCount() { + return sectionCount; + } + + /** + * A virtual writer for a tenant section within the HFile. + *

+ * This handles writing data for a specific tenant section as a complete HFile v3 structure. Each + * section maintains its own bloom filters and metadata while sharing the parent file's output + * stream through position translation. + */ + private class SectionWriter extends HFileWriterImpl { + /** The tenant section identifier for this section */ + private final byte[] tenantSectionId; + /** The starting offset of this section in the parent file */ + private final long sectionStartOffset; + /** Whether this section writer has been closed */ + private boolean closed = false; + /** Absolute offset where this section's load-on-open data begins */ + private long sectionLoadOnOpenOffset = -1L; + + /** + * Creates a section writer for a specific tenant section. + * @param conf Configuration settings + * @param cacheConf Cache configuration + * @param outputStream The parent file's output stream + * @param fileContext HFile context for this section + * @param tenantSectionId The tenant section identifier + * @param tenantId The tenant identifier for metadata + * @param sectionStartOffset Starting offset of this section + * @throws IOException if section writer creation fails + */ + public SectionWriter(Configuration conf, CacheConfig cacheConf, FSDataOutputStream outputStream, + HFileContext fileContext, byte[] tenantSectionId, byte[] tenantId, long sectionStartOffset) + throws IOException { + // Create a section-aware output stream that handles position translation + super(conf, cacheConf, null, + new SectionOutputStream(outputStream, sectionStartOffset, + MultiTenantHFileWriter.this.path != null + ? MultiTenantHFileWriter.this.path.getName() + : MultiTenantHFileWriter.this.streamName), + fileContext); + + this.tenantSectionId = tenantSectionId; + this.sectionStartOffset = sectionStartOffset; + + // Store the tenant ID in the file info + if (tenantId != null && tenantId.length > 0) { + appendFileInfo(Bytes.toBytes(FILEINFO_TENANT_ID), tenantId); + } + + // Store the section ID for reference + if (tenantSectionId != null) { + appendFileInfo(Bytes.toBytes(FILEINFO_TENANT_SECTION_ID), tenantSectionId); + } + + LOG.debug("Created section writer at offset {} for tenant section {}, tenant ID {}", + sectionStartOffset, + tenantSectionId == null ? "default" : Bytes.toStringBinary(tenantSectionId), + tenantId == null ? "default" : Bytes.toStringBinary(tenantId)); + } + + /** + * Output stream that translates positions relative to section start. + *

+ * This allows each section to maintain its own position tracking while writing to the shared + * parent file output stream. + */ + private static class SectionOutputStream extends FSDataOutputStream { + /** The delegate output stream (parent file stream) */ + private final FSDataOutputStream delegate; + /** The base offset of this section in the parent file */ + private final long baseOffset; + /** Logical file name used when the writer builds cache keys */ + private final String displayName; + + /** + * Creates a section-aware output stream. + * @param delegate The parent file's output stream + * @param baseOffset The starting offset of this section + */ + public SectionOutputStream(FSDataOutputStream delegate, long baseOffset, String displayName) { + super(delegate.getWrappedStream(), null); + this.delegate = delegate; + this.baseOffset = baseOffset; + this.displayName = displayName; + } + + @Override + public long getPos() { + try { + // Return position relative to section start + return delegate.getPos() - baseOffset; + } catch (Exception e) { + throw new RuntimeException("Failed to get position", e); + } + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + delegate.write(b, off, len); + } + + @Override + public void flush() throws IOException { + delegate.flush(); + } + + @Override + public void close() throws IOException { + // Don't close the delegate - it's shared across sections + flush(); + } + + @Override + public String toString() { + return displayName; + } + } + + @Override + public void append(ExtendedCell cell) throws IOException { + checkNotClosed(); + + super.append(cell); + } + + /** + * Safely handle adding general bloom filters to the section + */ + @Override + public void addGeneralBloomFilter(final BloomFilterWriter bfw) { + checkNotClosed(); + + // Skip empty bloom filters + if (bfw == null || bfw.getKeyCount() <= 0) { + LOG.debug("Skipping empty general bloom filter for tenant section: {}", + tenantSectionId == null ? "default" : Bytes.toStringBinary(tenantSectionId)); + return; + } + + // Ensure the bloom filter is properly initialized + bfw.compactBloom(); + + LOG.debug("Added general bloom filter with {} keys for tenant section: {}", bfw.getKeyCount(), + tenantSectionId == null ? "default" : Bytes.toStringBinary(tenantSectionId)); + + super.addGeneralBloomFilter(bfw); + } + + /** + * Safely handle adding delete family bloom filters to the section + */ + @Override + public void addDeleteFamilyBloomFilter(final BloomFilterWriter bfw) { + checkNotClosed(); + + // Skip empty bloom filters + if (bfw == null || bfw.getKeyCount() <= 0) { + LOG.debug("Skipping empty delete family bloom filter for tenant section: {}", + tenantSectionId == null ? "default" : Bytes.toStringBinary(tenantSectionId)); + return; + } + + // Ensure the bloom filter is properly initialized + bfw.compactBloom(); + + LOG.debug("Added delete family bloom filter with {} keys for tenant section: {}", + bfw.getKeyCount(), + tenantSectionId == null ? "default" : Bytes.toStringBinary(tenantSectionId)); + + // Call the parent implementation without try/catch since it doesn't actually throw + // IOException + // The HFileWriterImpl implementation doesn't throw IOException despite the interface + // declaration + super.addDeleteFamilyBloomFilter(bfw); + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + + LOG.debug("Closing section for tenant section ID: {}", + tenantSectionId == null ? "null" : Bytes.toStringBinary(tenantSectionId)); + + // Close the section writer safely + // HFileWriterImpl.close() can fail with NPE on empty bloom filters, but we want to + // still properly close the stream and resources + try { + super.close(); + } catch (RuntimeException e) { + LOG.warn("Error during section close, continuing with stream cleanup. Error: {}", + e.getMessage()); + // We will still mark as closed and continue with resource cleanup + } + closed = true; + + LOG.debug("Closed section for tenant section: {}", + tenantSectionId == null ? "default" : Bytes.toStringBinary(tenantSectionId)); + } + + /** + * Get the starting offset of this section in the file. + * @return The section's starting offset + */ + public long getSectionStartOffset() { + return sectionStartOffset; + } + + public long getSectionDataEndOffset() { + return sectionLoadOnOpenOffset; + } + + @Override + public Path getPath() { + // Return the parent file path + return MultiTenantHFileWriter.this.path; + } + + @Override + public void appendFileInfo(byte[] key, byte[] value) throws IOException { + checkNotClosed(); + super.appendFileInfo(key, value); + } + + @Override + public void appendMetaBlock(String metaBlockName, Writable content) { + checkNotClosed(); + super.appendMetaBlock(metaBlockName, content); + } + + @Override + public void addInlineBlockWriter(InlineBlockWriter ibw) { + checkNotClosed(); + super.addInlineBlockWriter(ibw); + } + + @Override + public void beforeShipped() throws IOException { + checkNotClosed(); + super.beforeShipped(); + } + + private void checkNotClosed() { + if (closed) { + throw new IllegalStateException("Section writer already closed"); + } + } + + // Override protected methods to make version 3 for each section + @Override + protected int getMajorVersion() { + return 3; // Each section uses version 3 format + } + + @Override + protected void finishClose(FixedFileTrailer trailer) throws IOException { + sectionLoadOnOpenOffset = sectionStartOffset + trailer.getLoadOnOpenDataOffset(); + super.finishClose(trailer); + } + + public long getTotalUncompressedBytes() { + return this.totalUncompressedBytes; + } + + /** + * Get the number of entries written to this section + * @return The entry count + */ + public long getEntryCount() { + return this.entryCount; + } + } + + /** + * An implementation of TenantExtractor that treats all data as belonging to a single default + * tenant. + *

+ * This extractor is used when multi-tenant functionality is disabled via the + * TABLE_MULTI_TENANT_ENABLED property set to false. It ensures that all cells are treated as + * belonging to the same tenant section, effectively creating a single-tenant HFile v4 with one + * section containing all data. + *

+ * Key characteristics: + *

    + *
  • Always returns the default empty tenant prefix for all cells
  • + *
  • Results in a single tenant section containing all data
  • + *
  • Maintains HFile v4 format compatibility while disabling multi-tenant features
  • + *
  • Useful for system tables or tables that don't require tenant isolation
  • + *
+ */ + static class SingleTenantExtractor implements TenantExtractor { + @Override + public byte[] extractTenantId(Cell cell) { + return DEFAULT_TENANT_PREFIX; + } + + @Override + public byte[] extractTenantSectionId(Cell cell) { + return DEFAULT_TENANT_PREFIX; + } + + @Override + public int getPrefixLength() { + return 0; + } + } + + /** + * Creates a specialized writer factory for multi-tenant HFiles format version 4. + *

+ * This factory automatically determines whether to create a multi-tenant or single-tenant writer + * based on table properties and configuration. It handles the extraction of table properties from + * the HFile context and applies proper configuration precedence. + */ + public static class WriterFactory extends HFile.WriterFactory { + /** Maintain our own copy of the file context */ + private HFileContext writerFileContext; + private BloomType preferredBloomType; + + /** + * Creates a new WriterFactory for multi-tenant HFiles. + * @param conf Configuration settings + * @param cacheConf Cache configuration + */ + public WriterFactory(Configuration conf, CacheConfig cacheConf) { + super(conf, cacheConf); + } + + @Override + public HFile.WriterFactory withFileContext(HFileContext fileContext) { + this.writerFileContext = fileContext; + return super.withFileContext(fileContext); + } + + public WriterFactory withPreferredBloomType(BloomType bloomType) { + this.preferredBloomType = bloomType; + return this; + } + + @Override + public HFile.Writer create() throws IOException { + if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) { + throw new AssertionError("Please specify exactly one of filesystem/path or path"); + } + + if (path != null) { + ostream = HFileWriterImpl.createOutputStream(conf, fs, path, favoredNodes); + try { + ostream.setDropBehind(shouldDropBehind && cacheConf.shouldDropBehindCompaction()); + } catch (UnsupportedOperationException uoe) { + LOG.trace("Unable to set drop behind on {}", path, uoe); + LOG.debug("Unable to set drop behind on {}", path.getName()); + } + } + + // Extract table properties for tenant configuration from table descriptor + Map tableProperties = new java.util.HashMap<>(); + BloomType columnFamilyBloomType = null; + + // Get the table descriptor if available + TableDescriptor tableDesc = getTableDescriptor(writerFileContext); + if (tableDesc != null) { + // Extract relevant properties for multi-tenant configuration + for (Entry entry : tableDesc.getValues().entrySet()) { + String key = Bytes.toString(entry.getKey().get()); + tableProperties.put(key, Bytes.toString(entry.getValue().get())); + } + columnFamilyBloomType = resolveColumnFamilyBloomType(tableDesc, writerFileContext); + LOG.debug( + "Creating MultiTenantHFileWriter with table properties from descriptor for table: {}", + tableDesc.getTableName()); + } else { + LOG.debug("Creating MultiTenantHFileWriter with default properties " + + "(no table descriptor available)"); + } + + // Create the writer using the factory method + // For system tables with MULTI_TENANT_ENABLED=false, this will use SingleTenantExtractor + // which creates HFile v4 with a single default section (clean and consistent) + // For user tables with multi-tenant properties, this will use DefaultTenantExtractor + // which creates HFile v4 with multiple tenant sections based on row key prefixes + boolean ownsStream = path != null; + return MultiTenantHFileWriter.create(fs, path, conf, cacheConf, tableProperties, + writerFileContext, columnFamilyBloomType, preferredBloomType, ostream, ownsStream); + } + + /** + * Get the table descriptor from the HFile context if available + * @param fileContext The HFile context potentially containing a table name + * @return The table descriptor or null if not available + */ + private TableDescriptor getTableDescriptor(HFileContext fileContext) { + try { + // If file context or table name is not available, return null + if (fileContext == null || fileContext.getTableName() == null) { + LOG.debug("Table name not available in HFileContext"); + return null; + } + + // Get the table descriptor from the Admin API + TableName tableName = TableName.valueOf(fileContext.getTableName()); + try (Connection conn = ConnectionFactory.createConnection(conf); + Admin admin = conn.getAdmin()) { + return admin.getDescriptor(tableName); + } catch (Exception e) { + LOG.warn("Failed to get table descriptor using Admin API for {}", tableName, e); + return null; + } + } catch (Exception e) { + LOG.warn("Error getting table descriptor", e); + return null; + } + } + + private BloomType resolveColumnFamilyBloomType(TableDescriptor tableDesc, + HFileContext fileContext) { + if (fileContext == null) { + return null; + } + + byte[] family = fileContext.getColumnFamily(); + if (family == null) { + return null; + } + + ColumnFamilyDescriptor familyDescriptor = tableDesc.getColumnFamily(family); + if (familyDescriptor == null) { + LOG.debug("Column family {} not found in table descriptor {}, using table-level bloom type", + Bytes.toStringBinary(family), tableDesc.getTableName()); + return null; + } + return familyDescriptor.getBloomFilterType(); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantPreadReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantPreadReader.java new file mode 100644 index 000000000000..8adf18c9e84c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantPreadReader.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * HFile reader for multi-tenant HFiles in PREAD (random access) mode. This implementation creates + * HFilePreadReader instances for each tenant section. + */ +@InterfaceAudience.Private +public class MultiTenantPreadReader extends AbstractMultiTenantReader { + private static final Logger LOG = LoggerFactory.getLogger(MultiTenantPreadReader.class); + + /** + * Constructor for multi-tenant pread reader. + * @param context Reader context info + * @param fileInfo HFile info + * @param cacheConf Cache configuration values + * @param conf Configuration + * @throws IOException If an error occurs during initialization + */ + public MultiTenantPreadReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf, + Configuration conf) throws IOException { + super(context, fileInfo, cacheConf, conf); + // Tenant index structure is loaded and logged by the parent class + } + + /** + * Create a section reader for a specific tenant. + *

+ * Creates a PreadSectionReader that handles positional read access to a specific tenant section + * within the multi-tenant HFile. + * @param tenantSectionId The tenant section ID + * @param metadata The section metadata containing offset and size + * @return A section reader for the tenant + * @throws IOException If an error occurs creating the reader + */ + @Override + protected SectionReader createSectionReader(byte[] tenantSectionId, SectionMetadata metadata) + throws IOException { + LOG.debug("Creating section reader for tenant section: {}, offset: {}, size: {}", + Bytes.toStringBinary(tenantSectionId), metadata.getOffset(), metadata.getSize()); + + return new PreadSectionReader(tenantSectionId, metadata); + } + + /** + * Section reader implementation for pread (positional read) access mode. + *

+ * This implementation creates HFilePreadReader instances for each tenant section, providing + * efficient random access to data within specific tenant boundaries. + */ + protected class PreadSectionReader extends SectionReader { + /** The underlying HFile reader for this section */ + private volatile HFileReaderImpl hFileReader; + + /** + * Constructor for PreadSectionReader. + * @param tenantSectionId The tenant section ID + * @param metadata The section metadata + */ + public PreadSectionReader(byte[] tenantSectionId, SectionMetadata metadata) { + super(tenantSectionId.clone(), metadata); + LOG.debug("Created PreadSectionReader for tenant section ID: {}", + Bytes.toStringBinary(this.tenantSectionId)); + } + + @Override + public HFileReaderImpl getReader() throws IOException { + HFileReaderImpl reader = hFileReader; + if (reader != null) { + return reader; + } + + synchronized (this) { + reader = hFileReader; + if (reader != null) { + return reader; + } + + try { + // Build section context with offset translation + ReaderContext sectionContext = + buildSectionContext(metadata, ReaderContext.ReaderType.PREAD); + + // Create unique file path for each section to enable proper prefetch scheduling + Path containerPath = sectionContext.getFilePath(); + String tenantSectionIdStr = Bytes.toStringBinary(tenantSectionId); + Path perSectionPath = new Path(containerPath.toString() + "#" + tenantSectionIdStr); + ReaderContext perSectionContext = + ReaderContextBuilder.newBuilder(sectionContext).withFilePath(perSectionPath).build(); + + // Create HFile info and reader for this section + HFileInfo info = new HFileInfo(perSectionContext, getConf()); + hFileReader = new HFilePreadReader(perSectionContext, info, cacheConf, getConf()); + + // Initialize metadata and indices + info.initMetaAndIndex(hFileReader); + + LOG.debug("Successfully initialized HFilePreadReader for tenant section ID: {}", + Bytes.toStringBinary(tenantSectionId)); + + return hFileReader; + } catch (IOException e) { + LOG.error("Failed to initialize section reader for tenant section at offset {}: {}", + metadata.getOffset(), e.getMessage()); + throw e; + } + } + } + + @Override + public HFileScanner getScanner(Configuration conf, boolean cacheBlocks, boolean pread, + boolean isCompaction) throws IOException { + HFileReaderImpl reader = getReader(); + HFileScanner scanner = reader.getScanner(conf, cacheBlocks, true, isCompaction); + LOG.debug( + "PreadSectionReader.getScanner for tenant section ID: {}, reader: {}, " + "scanner: {}", + Bytes.toStringBinary(tenantSectionId), reader, scanner); + return scanner; + } + + @Override + public void close(boolean evictOnClose) throws IOException { + if (hFileReader != null) { + hFileReader.close(evictOnClose); + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantReaderFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantReaderFactory.java new file mode 100644 index 000000000000..612e80bccc2e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantReaderFactory.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Factory for creating appropriate multi-tenant HFile readers based on the reader type. This + * handles both stream and pread access modes for multi-tenant HFiles. + */ +@InterfaceAudience.Private +public class MultiTenantReaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(MultiTenantReaderFactory.class); + + private MultiTenantReaderFactory() { + // Utility class, no instantiation + } + + /** + * Create the appropriate multi-tenant reader based on the reader type. + * @param context Reader context info + * @param fileInfo HFile info + * @param cacheConf Cache configuration values + * @param conf Configuration + * @return An appropriate multi-tenant HFile reader + * @throws IOException If an error occurs creating the reader + */ + public static HFile.Reader create(ReaderContext context, HFileInfo fileInfo, + CacheConfig cacheConf, Configuration conf) throws IOException { + + if (context.getReaderType() == ReaderContext.ReaderType.STREAM) { + LOG.debug("Creating MultiTenantStreamReader for {}", context.getFilePath()); + return new MultiTenantStreamReader(context, fileInfo, cacheConf, conf); + } else { + LOG.debug("Creating MultiTenantPreadReader for {}", context.getFilePath()); + return new MultiTenantPreadReader(context, fileInfo, cacheConf, conf); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantStreamReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantStreamReader.java new file mode 100644 index 000000000000..0455704726ed --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/MultiTenantStreamReader.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * HFile reader for multi-tenant HFiles in STREAM (sequential access) mode. This implementation + * creates HFileStreamReader instances for each tenant section. + */ +@InterfaceAudience.Private +public class MultiTenantStreamReader extends AbstractMultiTenantReader { + private static final Logger LOG = LoggerFactory.getLogger(MultiTenantStreamReader.class); + + /** + * Constructor for multi-tenant stream reader. + * @param context Reader context info + * @param fileInfo HFile info + * @param cacheConf Cache configuration values + * @param conf Configuration + * @throws IOException If an error occurs during initialization + */ + public MultiTenantStreamReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf, + Configuration conf) throws IOException { + super(context, fileInfo, cacheConf, conf); + // Tenant index structure is loaded and logged by the parent class + } + + /** + * Create a section reader for a specific tenant. + *

+ * Creates a StreamSectionReader that handles sequential access to a specific tenant section + * within the multi-tenant HFile. + * @param tenantSectionId The tenant section ID + * @param metadata The section metadata containing offset and size + * @return A section reader for the tenant + * @throws IOException If an error occurs creating the reader + */ + @Override + protected SectionReader createSectionReader(byte[] tenantSectionId, SectionMetadata metadata) + throws IOException { + LOG.debug("Creating section reader for tenant section: {}, offset: {}, size: {}", + Bytes.toStringBinary(tenantSectionId), metadata.getOffset(), metadata.getSize()); + return new StreamSectionReader(tenantSectionId, metadata); + } + + /** + * Section reader implementation for stream (sequential access) mode. + *

+ * This implementation creates HFileStreamReader instances for each tenant section, providing + * efficient sequential access to data within specific tenant boundaries. Stream readers are + * optimized for sequential scans and compaction operations. + */ + protected class StreamSectionReader extends SectionReader { + + /** + * Constructor for StreamSectionReader. + * @param tenantSectionId The tenant section ID + * @param metadata The section metadata + */ + public StreamSectionReader(byte[] tenantSectionId, SectionMetadata metadata) { + super(tenantSectionId, metadata); + } + + @Override + public synchronized HFileReaderImpl getReader() throws IOException { + if (!initialized) { + // Create section context with section-specific settings using parent method + ReaderContext sectionContext = + buildSectionContext(metadata, ReaderContext.ReaderType.STREAM); + + try { + // Create a section-specific HFileInfo + HFileInfo sectionFileInfo = new HFileInfo(sectionContext, getConf()); + + // Create stream reader for this section with the section-specific fileInfo + reader = new HFileStreamReader(sectionContext, sectionFileInfo, cacheConf, getConf()); + + // Initialize section indices using the standard HFileInfo method + // This method was designed for HFile v3 format, which each section follows + LOG.debug("Initializing section indices for tenant at offset {}", metadata.getOffset()); + sectionFileInfo.initMetaAndIndex(reader); + LOG.debug("Successfully initialized indices for section at offset {}", + metadata.getOffset()); + + initialized = true; + LOG.debug("Initialized HFileStreamReader for tenant section ID: {}", + org.apache.hadoop.hbase.util.Bytes.toStringBinary(tenantSectionId)); + } catch (IOException e) { + LOG.error("Failed to initialize section reader", e); + throw e; + } + } + return reader; + } + + @Override + public HFileScanner getScanner(Configuration conf, boolean cacheBlocks, boolean pread, + boolean isCompaction) throws IOException { + return getReader().getScanner(conf, cacheBlocks, pread, isCompaction); + } + + @Override + public void close(boolean evictOnClose) throws IOException { + if (reader != null) { + // Close underlying HFileStreamReader and unbuffer its wrapper + HFileReaderImpl r = reader; + reader = null; + r.close(evictOnClose); + r.getContext().getInputStreamWrapper().unbuffer(); + } + initialized = false; + } + } + + // No close overrides needed; inherited from AbstractMultiTenantReader +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/SectionIndexManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/SectionIndexManager.java new file mode 100644 index 000000000000..7416d5076987 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/SectionIndexManager.java @@ -0,0 +1,786 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages the section index for multi-tenant HFile version 4. This class contains both writer and + * reader functionality for section indices, which map tenant prefixes to file sections, allowing + * for efficient lookup of tenant-specific data in a multi-tenant HFile. + */ +@InterfaceAudience.Private +public class SectionIndexManager { + + private SectionIndexManager() { + // Utility class, no instantiation + } + + /** + * Default maximum number of entries in a single index block + */ + public static final int DEFAULT_MAX_CHUNK_SIZE = 128; + + /** + * Default minimum number of entries in the root index block + */ + public static final int DEFAULT_MIN_INDEX_NUM_ENTRIES = 16; + + /** + * Configuration key for maximum chunk size + */ + public static final String SECTION_INDEX_MAX_CHUNK_SIZE = "hbase.section.index.max.chunk.size"; + + /** + * Configuration key for minimum number of root entries + */ + public static final String SECTION_INDEX_MIN_NUM_ENTRIES = "hbase.section.index.min.num.entries"; + + /** + * Represents a tenant section entry in the index. + */ + public static class SectionIndexEntry { + /** The tenant prefix for this section */ + private final byte[] tenantPrefix; + /** The file offset where the section starts */ + private final long offset; + /** The size of the section in bytes */ + private final int sectionSize; + + /** + * Constructor for SectionIndexEntry. + * @param tenantPrefix the tenant prefix for this section + * @param offset the file offset where the section starts + * @param sectionSize the size of the section in bytes + */ + public SectionIndexEntry(byte[] tenantPrefix, long offset, int sectionSize) { + this.tenantPrefix = tenantPrefix; + this.offset = offset; + this.sectionSize = sectionSize; + } + + /** + * Get the tenant prefix for this section. + * @return the tenant prefix + */ + public byte[] getTenantPrefix() { + return tenantPrefix; + } + + /** + * Get the file offset where the section starts. + * @return the offset + */ + public long getOffset() { + return offset; + } + + /** + * Get the size of the section in bytes. + * @return the section size + */ + public int getSectionSize() { + return sectionSize; + } + + @Override + public String toString() { + return "SectionIndexEntry{" + "tenantPrefix=" + Bytes.toStringBinary(tenantPrefix) + + ", offset=" + offset + ", sectionSize=" + sectionSize + '}'; + } + } + + /** + * Represents a block in the multi-level section index. + */ + private static class SectionIndexBlock { + /** List of entries in this block */ + private final List entries = new ArrayList<>(); + /** The offset of this block in the file */ + private long blockOffset; + /** The size of this block in bytes */ + private int blockSize; + + /** + * Add an entry to this block. + * @param entry the entry to add + */ + public void addEntry(SectionIndexEntry entry) { + entries.add(entry); + } + + /** + * Get all entries in this block. + * @return the list of entries + */ + public List getEntries() { + return entries; + } + + /** + * Get the number of entries in this block. + * @return the entry count + */ + public int getEntryCount() { + return entries.size(); + } + + /** + * Get the first entry in this block. + * @return the first entry, or null if the block is empty + */ + public SectionIndexEntry getFirstEntry() { + return entries.isEmpty() ? null : entries.get(0); + } + + /** + * Set the metadata for this block. + * @param offset the offset of this block in the file + * @param size the size of this block in bytes + */ + public void setBlockMetadata(long offset, int size) { + this.blockOffset = offset; + this.blockSize = size; + } + + /** + * Get the offset of this block in the file. + * @return the block offset + */ + public long getBlockOffset() { + return blockOffset; + } + + /** + * Get the size of this block in bytes. + * @return the block size + */ + public int getBlockSize() { + return blockSize; + } + } + + /** + * Writer for section indices in multi-tenant HFile version 4. This writer collects section + * entries and writes them to the file as a multi-level index to support large tenant sets + * efficiently. + */ + public static class Writer { + private static final Logger LOG = LoggerFactory.getLogger(Writer.class); + + /** List of all section entries */ + private final List entries = new ArrayList<>(); + /** Block writer to use for index blocks */ + private final HFileBlock.Writer blockWriter; + /** Cache configuration (unused for section index blocks) */ + @SuppressWarnings("unused") + private final CacheConfig cacheConf; + /** File name to use for caching, or null if no caching (unused) */ + @SuppressWarnings("unused") + private final String nameForCaching; + + /** Maximum number of entries in a single index block */ + private int maxChunkSize = DEFAULT_MAX_CHUNK_SIZE; + /** Minimum number of entries in the root-level index block */ + private int minIndexNumEntries = DEFAULT_MIN_INDEX_NUM_ENTRIES; + /** Total uncompressed size of the index */ + private int totalUncompressedSize = 0; + /** Number of levels in this index */ + private int numLevels = 1; + + /** Track leaf blocks for building the multi-level index */ + private final List leafBlocks = new ArrayList<>(); + /** Track intermediate blocks for building the multi-level index */ + private final List intermediateBlocks = new ArrayList<>(); + + /** + * Constructor for Writer. + * @param blockWriter block writer to use for index blocks + * @param cacheConf cache configuration + * @param nameForCaching file name to use for caching, or null if no caching + */ + public Writer(HFileBlock.Writer blockWriter, CacheConfig cacheConf, String nameForCaching) { + this.blockWriter = blockWriter; + this.cacheConf = cacheConf; + this.nameForCaching = nameForCaching; + } + + /** + * Set the maximum number of entries in a single index block. + * @param maxChunkSize The maximum number of entries per block + */ + public void setMaxChunkSize(int maxChunkSize) { + this.maxChunkSize = maxChunkSize; + } + + /** + * Set the minimum number of entries in the root-level index block. + * @param minIndexNumEntries The minimum number of entries + */ + public void setMinIndexNumEntries(int minIndexNumEntries) { + this.minIndexNumEntries = minIndexNumEntries; + } + + /** + * Add a section entry to the index. + * @param tenantPrefix the tenant prefix for this section + * @param offset the file offset where the section starts + * @param sectionSize the size of the section in bytes + */ + public void addEntry(byte[] tenantPrefix, long offset, int sectionSize) { + SectionIndexEntry entry = new SectionIndexEntry( + tenantPrefix != null ? tenantPrefix : new byte[0], offset, sectionSize); + entries.add(entry); + + LOG.debug("Added section index entry: tenant={}, offset={}, size={}", + tenantPrefix != null ? Bytes.toStringBinary(tenantPrefix) : "default", offset, sectionSize); + } + + /** + * Helper to write a single section index entry (prefix, offset, size). + */ + private void writeEntry(DataOutputStream out, SectionIndexEntry entry) throws IOException { + byte[] prefix = entry.getTenantPrefix(); + out.writeInt(prefix.length); + out.write(prefix); + out.writeLong(entry.getOffset()); + out.writeInt(entry.getSectionSize()); + } + + /** + * Write the section index blocks to the output stream. For large tenant sets, this builds a + * multi-level index. + * @param outputStream the output stream to write to + * @return the offset where the section index root block starts + * @throws IOException if an I/O error occurs + */ + public long writeIndexBlocks(FSDataOutputStream outputStream) throws IOException { + // Handle empty indexes like HFileBlockIndex does - write valid empty structure + if (entries.isEmpty()) { + LOG.info("Writing empty section index (no tenant sections)"); + return writeEmptyIndex(outputStream); + } + + // Keep entries in their original order for sequential access + + // Determine if we need a multi-level index based on entry count + boolean multiLevel = entries.size() > maxChunkSize; + + // Clear any existing block tracking + leafBlocks.clear(); + intermediateBlocks.clear(); + + // For small indices, just write a single-level root block + if (!multiLevel) { + numLevels = 1; + return writeSingleLevelIndex(outputStream); + } + + // Split entries into leaf blocks + int numLeafBlocks = (entries.size() + maxChunkSize - 1) / maxChunkSize; + for (int blockIndex = 0; blockIndex < numLeafBlocks; blockIndex++) { + SectionIndexBlock block = new SectionIndexBlock(); + int startIndex = blockIndex * maxChunkSize; + int endIndex = Math.min((blockIndex + 1) * maxChunkSize, entries.size()); + + for (int entryIndex = startIndex; entryIndex < endIndex; entryIndex++) { + block.addEntry(entries.get(entryIndex)); + } + + leafBlocks.add(block); + } + + // Write leaf blocks + writeLeafBlocks(outputStream); + + // If we have few enough leaf blocks, root can point directly to them + if (leafBlocks.size() <= minIndexNumEntries) { + numLevels = 2; // Root + leaf level + return writeIntermediateBlock(outputStream, leafBlocks, true); + } + + // Otherwise, we need intermediate blocks + numLevels = 3; // Root + intermediate + leaf + + // Group leaf blocks into intermediate blocks + int intermediateBlocksNeeded = (leafBlocks.size() + maxChunkSize - 1) / maxChunkSize; + for (int blockIndex = 0; blockIndex < intermediateBlocksNeeded; blockIndex++) { + SectionIndexBlock block = new SectionIndexBlock(); + int startIndex = blockIndex * maxChunkSize; + int endIndex = Math.min((blockIndex + 1) * maxChunkSize, leafBlocks.size()); + + for (int leafIndex = startIndex; leafIndex < endIndex; leafIndex++) { + SectionIndexBlock leafBlock = leafBlocks.get(leafIndex); + // Add the first entry from this leaf block to the intermediate block + block.addEntry(leafBlock.getFirstEntry()); + } + + intermediateBlocks.add(block); + } + + // Write intermediate blocks + writeIntermediateBlocks(outputStream); + + // Write root block (pointing to intermediate blocks) + return writeIntermediateBlock(outputStream, intermediateBlocks, true); + } + + /** + * Write an empty index structure. This creates a valid but empty root block similar to how + * HFileBlockIndex handles empty indexes. + * @param out the output stream to write to + * @return the offset where the empty root block starts + * @throws IOException if an I/O error occurs + */ + private long writeEmptyIndex(FSDataOutputStream out) throws IOException { + // Record root offset + long rootOffset = out.getPos(); + + // Write empty root block + DataOutputStream dos = blockWriter.startWriting(BlockType.ROOT_INDEX); + dos.writeInt(0); // Zero entries + blockWriter.writeHeaderAndData(out); + + // Update metrics + totalUncompressedSize += blockWriter.getOnDiskSizeWithHeader(); + numLevels = 1; + + LOG.info("Wrote empty section index at offset {}", rootOffset); + + return rootOffset; + } + + /** + * Write a single-level index (just the root block). + */ + private long writeSingleLevelIndex(FSDataOutputStream out) throws IOException { + // Record root offset + long rootOffset = out.getPos(); + + // Write root block containing all entries + DataOutputStream dos = blockWriter.startWriting(BlockType.ROOT_INDEX); + writeRootBlock(dos, entries); + blockWriter.writeHeaderAndData(out); + + // Update metrics + totalUncompressedSize += blockWriter.getOnDiskSizeWithHeader(); + + LOG.info("Wrote single-level section index with {} entries at offset {}", entries.size(), + rootOffset); + + return rootOffset; + } + + /** + * Write all leaf-level blocks. + */ + private void writeLeafBlocks(FSDataOutputStream out) throws IOException { + for (SectionIndexBlock block : leafBlocks) { + // Write leaf block + long blockOffset = out.getPos(); + DataOutputStream dos = blockWriter.startWriting(BlockType.LEAF_INDEX); + writeIndexBlock(dos, block.getEntries()); + blockWriter.writeHeaderAndData(out); + + // Record block metadata for higher levels + block.setBlockMetadata(blockOffset, blockWriter.getOnDiskSizeWithHeader()); + + // Update metrics + totalUncompressedSize += blockWriter.getUncompressedSizeWithHeader(); + + LOG.debug("Wrote leaf section index block with {} entries at offset {}", + block.getEntryCount(), blockOffset); + } + } + + /** + * Write all intermediate-level blocks. + */ + private void writeIntermediateBlocks(FSDataOutputStream out) throws IOException { + for (int blockIndex = 0; blockIndex < intermediateBlocks.size(); blockIndex++) { + SectionIndexBlock block = intermediateBlocks.get(blockIndex); + long blockOffset = out.getPos(); + DataOutputStream dos = blockWriter.startWriting(BlockType.INTERMEDIATE_INDEX); + + int entryCount = block.getEntryCount(); + dos.writeInt(entryCount); + + // Entries in this intermediate block correspond to leaf blocks in range + // [startIndex, startIndex + entryCount) + int startIndex = blockIndex * maxChunkSize; + for (int i = 0; i < entryCount; i++) { + int leafIndex = startIndex + i; + SectionIndexBlock leafBlock = leafBlocks.get(leafIndex); + SectionIndexEntry firstEntry = leafBlock.getFirstEntry(); + + byte[] prefix = firstEntry.getTenantPrefix(); + dos.writeInt(prefix.length); + dos.write(prefix); + dos.writeLong(leafBlock.getBlockOffset()); + dos.writeInt(leafBlock.getBlockSize()); + } + + blockWriter.writeHeaderAndData(out); + + // Record block metadata for higher levels + block.setBlockMetadata(blockOffset, blockWriter.getOnDiskSizeWithHeader()); + + // Update metrics + totalUncompressedSize += blockWriter.getUncompressedSizeWithHeader(); + + LOG.debug("Wrote intermediate section index block with {} entries at offset {}", + block.getEntryCount(), blockOffset); + } + } + + /** + * Write an intermediate or root block that points to other blocks. + */ + private long writeIntermediateBlock(FSDataOutputStream out, List blocks, + boolean isRoot) throws IOException { + long blockOffset = out.getPos(); + DataOutputStream dos = + blockWriter.startWriting(isRoot ? BlockType.ROOT_INDEX : BlockType.INTERMEDIATE_INDEX); + + // Write block count + dos.writeInt(blocks.size()); + + // Write entries using helper + block metadata + for (SectionIndexBlock block : blocks) { + SectionIndexEntry firstEntry = block.getFirstEntry(); + writeEntry(dos, firstEntry); + dos.writeLong(block.getBlockOffset()); + dos.writeInt(block.getBlockSize()); + } + + blockWriter.writeHeaderAndData(out); + + // Update metrics + totalUncompressedSize += blockWriter.getUncompressedSizeWithHeader(); + + LOG.debug("Wrote {} section index block with {} entries at offset {}", + isRoot ? "root" : "intermediate", blocks.size(), blockOffset); + + return blockOffset; + } + + /** + * Write a standard index block with section entries. + */ + private void writeIndexBlock(DataOutputStream out, List blockEntries) + throws IOException { + // Write entry count + out.writeInt(blockEntries.size()); + + // Write each entry using helper + for (SectionIndexEntry entry : blockEntries) { + writeEntry(out, entry); + } + } + + /** + * Write a root block. + */ + private void writeRootBlock(DataOutputStream out, List entries) + throws IOException { + // Just delegate to the standard index block writer + writeIndexBlock(out, entries); + } + + /** + * Get the number of root entries in the index. + * @return the number of entries at the root level + */ + public int getNumRootEntries() { + if (numLevels == 1) { + return entries.size(); + } else if (numLevels == 2) { + return leafBlocks.size(); + } else { + return intermediateBlocks.size(); + } + } + + /** + * Get the number of levels in this index. + * @return the number of levels (1 for single level, 2+ for multi-level) + */ + public int getNumLevels() { + return numLevels; + } + + /** + * Get the total uncompressed size of the index. + * @return the total uncompressed size in bytes + */ + public int getTotalUncompressedSize() { + return totalUncompressedSize; + } + + /** + * Clear all entries from the index. + */ + public void clear() { + entries.clear(); + leafBlocks.clear(); + intermediateBlocks.clear(); + totalUncompressedSize = 0; + numLevels = 1; + } + } + + /** + * Reader for section indices in multi-tenant HFile version 4. Supports both single-level and + * multi-level indices. + */ + public static class Reader { + private static final Logger LOG = LoggerFactory.getLogger(Reader.class); + + /** List of all section entries loaded from the index */ + private final List sections = new ArrayList<>(); + /** Number of levels in the loaded index */ + private int numLevels = 1; + + /** + * Default constructor for Reader. + */ + public Reader() { + // Empty constructor + } + + /** + * Load a section index from an HFile block. + * @param block the HFile block containing the section index + * @throws IOException if an I/O error occurs + */ + public void loadSectionIndex(HFileBlock block) throws IOException { + if (block.getBlockType() != BlockType.ROOT_INDEX) { + throw new IOException( + "Block is not a ROOT_INDEX for section index: " + block.getBlockType()); + } + + sections.clear(); + DataInputStream in = block.getByteStream(); + + try { + // Read the number of sections + int numSections = in.readInt(); + + // Read each section entry + for (int i = 0; i < numSections; i++) { + // Read tenant prefix + int prefixLength = in.readInt(); + byte[] prefix = new byte[prefixLength]; + in.readFully(prefix); + + // Read offset and size + long offset = in.readLong(); + int size = in.readInt(); + + // Add the entry + sections.add(new SectionIndexEntry(prefix, offset, size)); + } + + LOG.debug("Loaded section index with {} entries", sections.size()); + } catch (IOException e) { + LOG.error("Failed to load section index", e); + sections.clear(); + throw e; + } + } + + /** + * Load a (potentially multi-level) section index from the given root index block. This API + * requires the number of index levels (from the trailer) and an FS reader for fetching + * intermediate/leaf blocks when needed. + * @param rootBlock the ROOT_INDEX block where the section index starts + * @param levels the number of index levels; 1 for single-level, >=2 for multi-level + * @param fsReader the filesystem block reader to fetch child index blocks + */ + public void loadSectionIndex(HFileBlock rootBlock, int levels, HFileBlock.FSReader fsReader) + throws IOException { + if (rootBlock.getBlockType() != BlockType.ROOT_INDEX) { + throw new IOException( + "Block is not a ROOT_INDEX for section index: " + rootBlock.getBlockType()); + } + if (levels < 1) { + throw new IOException("Invalid index level count: " + levels); + } + sections.clear(); + this.numLevels = levels; + + if (levels == 1) { + // Single-level index: entries are directly in the root + loadSectionIndex(rootBlock); + return; + } + + if (fsReader == null) { + throw new IOException("FSReader is required to read multi-level section index"); + } + + // Multi-level: root contains pointers to next-level blocks. + DataInputStream in = rootBlock.getByteStream(); + int fanout = in.readInt(); + if (fanout < 0) { + throw new IOException("Negative root entry count in section index: " + fanout); + } + for (int i = 0; i < fanout; i++) { + // Root entry: first leaf entry (prefix, offset, size) + child pointer (offset, size) + int prefixLength = in.readInt(); + byte[] prefix = new byte[prefixLength]; + in.readFully(prefix); + in.readLong(); // first entry offset (ignored) + in.readInt(); // first entry size (ignored) + long childBlockOffset = in.readLong(); + int childBlockSize = in.readInt(); + + readChildIndexSubtree(childBlockOffset, childBlockSize, levels - 1, fsReader); + } + + LOG.debug("Loaded multi-level section index: levels={}, sections={}", this.numLevels, + sections.size()); + } + + /** + * Recursively read intermediate/leaf index blocks and collect section entries. + */ + private void readChildIndexSubtree(long blockOffset, int blockSize, int levelsRemaining, + HFileBlock.FSReader fsReader) throws IOException { + HFileBlock child = fsReader.readBlockData(blockOffset, blockSize, true, true, true); + HFileBlock blockToRead = null; + try { + blockToRead = child.unpack(child.getHFileContext(), fsReader); + if (levelsRemaining == 1) { + // Leaf level: contains actual section entries + if (blockToRead.getBlockType() != BlockType.LEAF_INDEX) { + LOG.warn("Expected LEAF_INDEX at leaf level but found {}", blockToRead.getBlockType()); + } + readLeafBlock(blockToRead); + return; + } + + // Intermediate level: each entry points to a child block + if (blockToRead.getBlockType() != BlockType.INTERMEDIATE_INDEX) { + LOG.warn("Expected INTERMEDIATE_INDEX at level {} but found {}", levelsRemaining, + blockToRead.getBlockType()); + } + DataInputStream in = blockToRead.getByteStream(); + int entryCount = in.readInt(); + if (entryCount < 0) { + throw new IOException( + "Negative intermediate entry count in section index: " + entryCount); + } + for (int i = 0; i < entryCount; i++) { + int prefixLength = in.readInt(); + byte[] prefix = new byte[prefixLength]; + in.readFully(prefix); + long nextOffset = in.readLong(); + int nextSize = in.readInt(); + readChildIndexSubtree(nextOffset, nextSize, levelsRemaining - 1, fsReader); + } + } finally { + // Release as these are non-root, transient blocks + if (blockToRead != null && blockToRead != child) { + try { + blockToRead.release(); + } catch (Throwable t) { + // ignore + } + } + try { + child.release(); + } catch (Throwable t) { + // ignore + } + } + } + + /** + * Parse a leaf index block and append all section entries. + */ + private void readLeafBlock(HFileBlock leafBlock) throws IOException { + DataInputStream in = leafBlock.getByteStream(); + int num = in.readInt(); + if (num < 0) { + throw new IOException("Negative leaf entry count in section index: " + num); + } + for (int i = 0; i < num; i++) { + int prefixLength = in.readInt(); + byte[] prefix = new byte[prefixLength]; + in.readFully(prefix); + long offset = in.readLong(); + int size = in.readInt(); + sections.add(new SectionIndexEntry(prefix, offset, size)); + } + } + + /** + * Find the section entry for a given tenant prefix. + * @param tenantPrefix the tenant prefix to look up + * @return the section entry, or null if not found + */ + public SectionIndexEntry findSection(byte[] tenantPrefix) { + for (SectionIndexEntry entry : sections) { + if (Bytes.equals(entry.getTenantPrefix(), tenantPrefix)) { + return entry; + } + } + return null; + } + + /** + * Get all section entries in the index. + * @return the list of section entries + */ + public List getSections() { + return new ArrayList<>(sections); + } + + /** + * Get the number of sections in the index. + * @return the number of sections + */ + public int getNumSections() { + return sections.size(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("SectionIndexReader{sections=").append(sections.size()).append(", entries=["); + for (int i = 0; i < sections.size(); i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(sections.get(i)); + } + sb.append("]}"); + return sb.toString(); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TenantExtractor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TenantExtractor.java new file mode 100644 index 000000000000..d789636e53b4 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TenantExtractor.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import org.apache.hadoop.hbase.Cell; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Strategy interface for extracting tenant information from cells following SOLID's Interface + * Segregation Principle. + */ +@InterfaceAudience.Private +public interface TenantExtractor { + /** + * Extract tenant ID from a cell + * @param cell The cell to extract tenant information from + * @return The tenant ID as a byte array + */ + byte[] extractTenantId(Cell cell); + + /** + * Extract tenant section ID from a cell for use in section index blocks + * @param cell The cell to extract tenant section information from + * @return The tenant section ID as a byte array + */ + byte[] extractTenantSectionId(Cell cell); + + /** + * Get the tenant prefix length used for extraction + * @return The length of the tenant prefix in bytes + */ + int getPrefixLength(); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TenantExtractorFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TenantExtractorFactory.java new file mode 100644 index 000000000000..10b4bd8e5568 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TenantExtractorFactory.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Factory for creating TenantExtractor instances based on configuration. + *

+ * Tenant configuration is obtained from cluster configuration and table properties, not from + * HFileContext. + *

+ * For HFile v4, tenant configuration is stored in the file trailer, allowing it to be accessed + * before the file info blocks are loaded. This resolves timing issues in the reader initialization + * process. + */ +@InterfaceAudience.Private +public class TenantExtractorFactory { + /** Logger for this class */ + private static final Logger LOG = LoggerFactory.getLogger(TenantExtractorFactory.class); + + private TenantExtractorFactory() { + // Utility class, no instantiation + } + + /** Default tenant prefix length when not specified in configuration */ + private static final int DEFAULT_PREFIX_LENGTH = 4; + + /** + * Create a TenantExtractor from HFile's reader context. This method is called during HFile + * reading to determine how to extract tenant information. + * @param reader The HFile reader that contains file info + * @return Appropriate TenantExtractor implementation + */ + public static TenantExtractor createFromReader(HFile.Reader reader) { + // Check if this is a v4 file with tenant configuration in the trailer + FixedFileTrailer trailer = reader.getTrailer(); + if (trailer.getMajorVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT) { + if (trailer.isMultiTenant()) { + int prefixLength = trailer.getTenantPrefixLength(); + LOG.info("Multi-tenant enabled from HFile v4 trailer, prefixLength={}", prefixLength); + return new DefaultTenantExtractor(prefixLength); + } else { + LOG.info("HFile v4 format, but multi-tenant not enabled in trailer"); + return new MultiTenantHFileWriter.SingleTenantExtractor(); + } + } + + // For non-v4 files, always use SingleTenantExtractor + LOG.info("Non-v4 HFile format (v{}), using SingleTenantExtractor", trailer.getMajorVersion()); + return new MultiTenantHFileWriter.SingleTenantExtractor(); + } + + /** + * Create a tenant extractor based on configuration. This applies configuration with proper + * precedence: 1. Table level settings have highest precedence 2. Cluster level settings are used + * as fallback 3. Default values are used if neither is specified + * @param conf HBase configuration for cluster defaults + * @param tableProperties Table properties for table-specific settings + * @return A configured TenantExtractor + */ + public static TenantExtractor createTenantExtractor(Configuration conf, + Map tableProperties) { + + // Check if multi-tenant functionality is enabled for this table + boolean multiTenantEnabled = false; // Default to disabled - only enabled when explicitly set + if ( + tableProperties != null + && tableProperties.containsKey(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED) + ) { + multiTenantEnabled = Boolean + .parseBoolean(tableProperties.get(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED)); + } + + // If multi-tenant is disabled, return SingleTenantExtractor + if (!multiTenantEnabled) { + LOG.info("Multi-tenant functionality disabled for this table, using SingleTenantExtractor"); + return new MultiTenantHFileWriter.SingleTenantExtractor(); + } + + // Multi-tenant enabled - configure DefaultTenantExtractor + + // First try table level settings (highest precedence) + String tablePrefixLengthStr = tableProperties != null + ? tableProperties.get(MultiTenantHFileWriter.TABLE_TENANT_PREFIX_LENGTH) + : null; + + // If not found at table level, try cluster level settings + int clusterPrefixLength = + conf.getInt(MultiTenantHFileWriter.TENANT_PREFIX_LENGTH, DEFAULT_PREFIX_LENGTH); + + // Use table settings if available, otherwise use cluster settings + int prefixLength; + if (tablePrefixLengthStr != null) { + try { + prefixLength = Integer.parseInt(tablePrefixLengthStr); + } catch (NumberFormatException nfe) { + LOG.warn("Invalid table-level tenant prefix length '{}', using cluster default {}", + tablePrefixLengthStr, clusterPrefixLength); + prefixLength = clusterPrefixLength; + } + } else { + prefixLength = clusterPrefixLength; + } + + LOG.info("Tenant configuration initialized: prefixLength={}, from table properties: {}", + prefixLength, (tablePrefixLengthStr != null)); + + // Create and return a DefaultTenantExtractor with the configured parameters + return new DefaultTenantExtractor(prefixLength); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java index c7c4801c6595..429c6246a17a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java @@ -68,7 +68,17 @@ public StoreFileScanner getScanner() throws IOException { * @return The cell in the mob file. */ public MobCell readCell(ExtendedCell search, boolean cacheMobBlocks) throws IOException { - return readCell(search, cacheMobBlocks, sf.getMaxMemStoreTS()); + if (sf == null) { + throw new IOException("Mob file reader has been closed"); + } + sf.initReader(); + long readPoint = sf.getMaxMemStoreTS(); + if (readPoint < 0) { + // Reader metadata (including MAX_MEMSTORE_TS_KEY) is loaded only after initReader(). + // Fall back to disabling MVCC filtering when metadata is unavailable (e.g., legacy files). + readPoint = Long.MAX_VALUE; + } + return readCell(search, cacheMobBlocks, readPoint); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java index d2b88a501ec5..9fc024a08492 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java @@ -26,7 +26,6 @@ import java.util.function.Function; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ExtendedCell; -import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -56,8 +55,8 @@ public void append(ExtendedCell cell) throws IOException { timeRangeTracker.setMin(tieringValue); timeRangeTracker.setMax(tieringValue); lowerBoundary2TimeRanger.put(entry.getKey(), timeRangeTracker); - ((HFileWriterImpl) lowerBoundary2Writer.get(entry.getKey()).getLiveFileWriter()) - .setTimeRangeTrackerForTiering(() -> timeRangeTracker); + lowerBoundary2Writer.get(entry.getKey()) + .setCustomTieringTimeRangeSupplier(() -> timeRangeTracker); } else { TimeRangeTracker timeRangeTracker = entry.getValue(); if (timeRangeTracker.getMin() > tieringValue) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index a7df71f460e4..bd6a19f98548 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -397,7 +397,14 @@ private void open() throws IOException { StoreFileReader reader = fileInfo.preStoreFileReaderOpen(context, cacheConf); if (reader == null) { reader = fileInfo.createReader(context, cacheConf); - fileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader()); + // Only initialize meta and index for non-multi-tenant files (v3 and below) + // Multi-tenant files (v4) skip this initialization just like in HFile.createReader() + if ( + fileInfo.getHFileInfo().getTrailer().getMajorVersion() + != HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT + ) { + fileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader()); + } } this.initialReader = fileInfo.postStoreFileReaderOpen(context, cacheConf, reader); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java index 25a16b1d8882..56f483952a23 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileInfo; import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.io.hfile.MultiTenantBloomSupport; import org.apache.hadoop.hbase.io.hfile.ReaderContext; import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -88,6 +89,12 @@ public class StoreFileReader { private final StoreFileInfo storeFileInfo; private final ReaderContext context; + private void incrementBloomEligible() { + if (bloomFilterMetrics != null) { + bloomFilterMetrics.incrementEligible(); + } + } + private StoreFileReader(HFile.Reader reader, StoreFileInfo storeFileInfo, ReaderContext context, Configuration conf) { this.reader = reader; @@ -249,13 +256,22 @@ boolean passesBloomFilter(Scan scan, final SortedSet columns) { return passesGeneralRowPrefixBloomFilter(scan); default: if (scan.isGetScan()) { - bloomFilterMetrics.incrementEligible(); + incrementBloomEligible(); } return true; } } public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowLen) { + if (reader instanceof MultiTenantBloomSupport) { + try { + return ((MultiTenantBloomSupport) reader).passesDeleteFamilyBloomFilter(row, rowOffset, + rowLen); + } catch (IOException e) { + LOG.warn("Failed multi-tenant delete-family bloom check, proceeding without", e); + return true; + } + } // Cache Bloom filter as a local variable in case it is set to null by // another thread on an IO error. BloomFilter bloomFilter = this.deleteFamilyBloomFilter; @@ -288,9 +304,19 @@ public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowL * @return True if passes */ private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) { + if (reader instanceof MultiTenantBloomSupport) { + incrementBloomEligible(); + try { + return ((MultiTenantBloomSupport) reader).passesGeneralRowBloomFilter(row, rowOffset, + rowLen); + } catch (IOException e) { + LOG.warn("Failed multi-tenant row bloom check, proceeding without", e); + return true; + } + } BloomFilter bloomFilter = this.generalBloomFilter; if (bloomFilter == null) { - bloomFilterMetrics.incrementEligible(); + incrementBloomEligible(); return true; } @@ -309,9 +335,18 @@ private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLe * @return True if passes */ public boolean passesGeneralRowColBloomFilter(ExtendedCell cell) { + if (reader instanceof MultiTenantBloomSupport) { + incrementBloomEligible(); + try { + return ((MultiTenantBloomSupport) reader).passesGeneralRowColBloomFilter(cell); + } catch (IOException e) { + LOG.warn("Failed multi-tenant row/col bloom check, proceeding without", e); + return true; + } + } BloomFilter bloomFilter = this.generalBloomFilter; if (bloomFilter == null) { - bloomFilterMetrics.incrementEligible(); + incrementBloomEligible(); return true; } // Used in ROW_COL bloom @@ -333,7 +368,7 @@ public boolean passesGeneralRowColBloomFilter(ExtendedCell cell) { private boolean passesGeneralRowPrefixBloomFilter(Scan scan) { BloomFilter bloomFilter = this.generalBloomFilter; if (bloomFilter == null) { - bloomFilterMetrics.incrementEligible(); + incrementBloomEligible(); return true; } @@ -463,18 +498,41 @@ public Map loadFileInfo() throws IOException { bloomFilterType = BloomType.valueOf(Bytes.toString(b)); } + if (bloomFilterType == BloomType.NONE && reader instanceof MultiTenantBloomSupport) { + bloomFilterType = ((MultiTenantBloomSupport) reader).getGeneralBloomFilterType(); + } + byte[] p = fi.get(BLOOM_FILTER_PARAM_KEY); - if (bloomFilterType == BloomType.ROWPREFIX_FIXED_LENGTH) { + if (p != null) { prefixLength = Bytes.toInt(p); + } else if (reader instanceof MultiTenantBloomSupport) { + try { + prefixLength = ((MultiTenantBloomSupport) reader).getGeneralBloomPrefixLength(); + } catch (IOException e) { + LOG.debug("Failed to obtain prefix length from multi-tenant reader", e); + } } lastBloomKey = fi.get(LAST_BLOOM_KEY); - if (bloomFilterType == BloomType.ROWCOL) { + if (lastBloomKey == null && reader instanceof MultiTenantBloomSupport) { + try { + lastBloomKey = ((MultiTenantBloomSupport) reader).getLastBloomKey(); + } catch (IOException e) { + LOG.debug("Failed to obtain last bloom key from multi-tenant reader", e); + } + } + if (bloomFilterType == BloomType.ROWCOL && lastBloomKey != null) { lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length); } byte[] cnt = fi.get(DELETE_FAMILY_COUNT); if (cnt != null) { deleteFamilyCnt = Bytes.toLong(cnt); + } else if (reader instanceof MultiTenantBloomSupport) { + try { + deleteFamilyCnt = ((MultiTenantBloomSupport) reader).getDeleteFamilyBloomCount(); + } catch (IOException e) { + LOG.debug("Failed to obtain delete family bloom count from multi-tenant reader", e); + } } return fi; @@ -506,6 +564,12 @@ public void loadBloomfilter(BlockType blockType, BloomFilterMetrics metrics) { + reader.getName()); } } + } else if (reader instanceof MultiTenantBloomSupport) { + try { + generalBloomFilter = ((MultiTenantBloomSupport) reader).getGeneralBloomFilterInstance(); + } catch (IOException e) { + LOG.debug("Failed to obtain general bloom filter from multi-tenant reader", e); + } } } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) { if (this.deleteFamilyBloomFilter != null) return; // Bloom has been loaded @@ -518,6 +582,13 @@ public void loadBloomfilter(BlockType blockType, BloomFilterMetrics metrics) { LOG.info( "Loaded Delete Family Bloom (" + deleteFamilyBloomFilter.getClass().getSimpleName() + ") metadata for " + reader.getName()); + } else if (reader instanceof MultiTenantBloomSupport) { + try { + deleteFamilyBloomFilter = + ((MultiTenantBloomSupport) reader).getDeleteFamilyBloomFilterInstance(); + } catch (IOException e) { + LOG.debug("Failed to obtain delete family bloom filter from multi-tenant reader", e); + } } } else { throw new RuntimeException( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index c2f3705fa631..6ef0f1430d21 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; +import org.apache.hadoop.hbase.io.hfile.MultiTenantHFileWriter; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.util.BloomContext; @@ -264,6 +265,20 @@ public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTrack } } + /** + * Registers a supplier that exposes the custom tiering {@link TimeRangeTracker}. Concrete + * {@link HFile.Writer} implementations can use it to tune caching decisions or emit metadata. + */ + public void setCustomTieringTimeRangeSupplier(Supplier supplier) { + if (supplier == null) { + return; + } + liveFileWriter.setCustomTieringTimeRangeSupplier(supplier); + if (historicalFileWriter != null) { + historicalFileWriter.setCustomTieringTimeRangeSupplier(supplier); + } + } + @Override public void beforeShipped() throws IOException { liveFileWriter.beforeShipped(); @@ -312,6 +327,72 @@ public void appendFileInfo(byte[] key, byte[] value) throws IOException { } } + /** + * Thread compaction context into the writer so downstream formats (e.g., v4 sections) can reflect + * MAJOR_COMPACTION_KEY/HISTORICAL/COMPACTION_EVENT_KEY consistently. Should be called immediately + * after writer creation and before any cells are appended. + *

+ * Effects: + *

    + *
  • Writes {@link HStoreFile#MAJOR_COMPACTION_KEY} to indicate major/minor compaction.
  • + *
  • Writes {@link HStoreFile#COMPACTION_EVENT_KEY} built from the compaction input set. See + * {@link #buildCompactionEventTrackerBytes(java.util.function.Supplier, java.util.Collection)} + * for inclusion semantics.
  • + *
  • Writes {@link HStoreFile#HISTORICAL_KEY}: {@code false} for the live writer and + * {@code true} for the historical writer (when dual-writing is enabled).
  • + *
+ * For HFile v4 (multi-tenant) writers, these file info entries are propagated to each newly + * created tenant section so that every section reflects the real compaction context. + * @param majorCompaction {@code true} if this compaction is major, otherwise {@code false} + * @param storeFiles the set of input store files being compacted into this writer + * @throws IOException if writing file info fails + */ + public void appendCompactionContext(final boolean majorCompaction, + final Collection storeFiles) throws IOException { + byte[] eventBytes = buildCompactionEventTrackerBytes(this.compactedFilesSupplier, storeFiles); + // live file + liveFileWriter.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); + liveFileWriter.appendFileInfo(COMPACTION_EVENT_KEY, eventBytes); + liveFileWriter.appendFileInfo(HISTORICAL_KEY, Bytes.toBytes(false)); + // historical file if enabled + if (historicalFileWriter != null) { + historicalFileWriter.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); + historicalFileWriter.appendFileInfo(COMPACTION_EVENT_KEY, eventBytes); + historicalFileWriter.appendFileInfo(HISTORICAL_KEY, Bytes.toBytes(true)); + } + } + + /** + * Used when write {@link HStoreFile#COMPACTION_EVENT_KEY} to new file's file info. The compacted + * store files' names are needed. If a compacted store file is itself the result of compaction, + * its compacted files which are still not archived are needed, too. We do not add compacted files + * recursively. + *

+ * Example: If files A, B, C compacted to new file D, and file D compacted to new file E, we will + * write A, B, C, D to file E's compacted files. If later file E compacted to new file F, we will + * add E to F's compacted files first, then add E's compacted files (A, B, C, D) to it. There is + * no need to add D's compacted file again, as D's compacted files have already been included in + * E's compacted files. See HBASE-20724 for more details. + * @param compactedFilesSupplier supplier returning store files compacted but not yet archived + * @param storeFiles the compacted store files to generate this new file + * @return bytes of CompactionEventTracker + */ + private static byte[] buildCompactionEventTrackerBytes( + Supplier> compactedFilesSupplier, Collection storeFiles) { + Set notArchivedCompactedStoreFiles = compactedFilesSupplier.get().stream() + .map(sf -> sf.getPath().getName()).collect(Collectors.toSet()); + Set compactedStoreFiles = new HashSet<>(); + for (HStoreFile storeFile : storeFiles) { + compactedStoreFiles.add(storeFile.getFileInfo().getPath().getName()); + for (String csf : storeFile.getCompactedStoreFiles()) { + if (notArchivedCompactedStoreFiles.contains(csf)) { + compactedStoreFiles.add(csf); + } + } + } + return ProtobufUtil.toCompactionEventTrackerBytes(compactedStoreFiles); + } + /** * For use in testing. */ @@ -501,6 +582,7 @@ public void append(ExtendedCell cell) throws IOException { private static class SingleStoreFileWriter { private final BloomFilterWriter generalBloomFilterWriter; private final BloomFilterWriter deleteFamilyBloomFilterWriter; + private final boolean multiTenantWriter; private final BloomType bloomType; private byte[] bloomParam = null; private long deleteFamilyCnt = 0; @@ -530,58 +612,78 @@ private SingleStoreFileWriter(FileSystem fs, Path path, final Configuration conf Supplier> compactedFilesSupplier) throws IOException { this.compactedFilesSupplier = compactedFilesSupplier; // TODO : Change all writers to be specifically created for compaction context - writer = - HFile.getWriterFactory(conf, cacheConf).withPath(fs, path).withFavoredNodes(favoredNodes) - .withFileContext(fileContext).withShouldDropCacheBehind(shouldDropCacheBehind).create(); + HFile.WriterFactory writerFactory = HFile.getWriterFactory(conf, cacheConf); + if ( + writerFactory instanceof org.apache.hadoop.hbase.io.hfile.MultiTenantHFileWriter.WriterFactory + ) { + ((org.apache.hadoop.hbase.io.hfile.MultiTenantHFileWriter.WriterFactory) writerFactory) + .withPreferredBloomType(bloomType); + } + writer = writerFactory.withPath(fs, path).withFavoredNodes(favoredNodes) + .withFileContext(fileContext).withShouldDropCacheBehind(shouldDropCacheBehind).create(); - generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf, - bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); + this.multiTenantWriter = + writer instanceof org.apache.hadoop.hbase.io.hfile.MultiTenantHFileWriter; - if (generalBloomFilterWriter != null) { + if (multiTenantWriter) { + // Multi-tenant writer manages per-section bloom filters internally. + this.generalBloomFilterWriter = null; + this.deleteFamilyBloomFilterWriter = null; this.bloomType = bloomType; - this.bloomParam = BloomFilterUtil.getBloomFilterParam(bloomType, conf); - if (LOG.isTraceEnabled()) { - LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: " - + (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH - ? Bytes.toInt(bloomParam) - : Bytes.toStringBinary(bloomParam)) - + ", " + generalBloomFilterWriter.getClass().getSimpleName()); - } - // init bloom context - switch (bloomType) { - case ROW: - bloomContext = - new RowBloomContext(generalBloomFilterWriter, fileContext.getCellComparator()); - break; - case ROWCOL: - bloomContext = - new RowColBloomContext(generalBloomFilterWriter, fileContext.getCellComparator()); - break; - case ROWPREFIX_FIXED_LENGTH: - bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter, - fileContext.getCellComparator(), Bytes.toInt(bloomParam)); - break; - default: - throw new IOException( - "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL or ROWPREFIX expected)"); - } + this.bloomParam = null; } else { - // Not using Bloom filters. - this.bloomType = BloomType.NONE; - } + generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf, + bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); + + if (generalBloomFilterWriter != null) { + this.bloomType = bloomType; + this.bloomParam = BloomFilterUtil.getBloomFilterParam(bloomType, conf); + if (LOG.isTraceEnabled()) { + LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: " + + (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH + ? Bytes.toInt(bloomParam) + : Bytes.toStringBinary(bloomParam)) + + ", " + generalBloomFilterWriter.getClass().getSimpleName()); + } + // init bloom context + switch (bloomType) { + case ROW: + bloomContext = + new RowBloomContext(generalBloomFilterWriter, fileContext.getCellComparator()); + break; + case ROWCOL: + bloomContext = + new RowColBloomContext(generalBloomFilterWriter, fileContext.getCellComparator()); + break; + case ROWPREFIX_FIXED_LENGTH: + bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter, + fileContext.getCellComparator(), Bytes.toInt(bloomParam)); + break; + default: + throw new IOException("Invalid Bloom filter type: " + bloomType + + " (ROW or ROWCOL or ROWPREFIX expected)"); + } + } else { + // Not using Bloom filters. + this.bloomType = BloomType.NONE; + this.bloomParam = null; + } - // initialize delete family Bloom filter when there is NO RowCol Bloom filter - if (this.bloomType != BloomType.ROWCOL) { - this.deleteFamilyBloomFilterWriter = BloomFilterFactory.createDeleteBloomAtWrite(conf, - cacheConf, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); - deleteFamilyBloomContext = - new RowBloomContext(deleteFamilyBloomFilterWriter, fileContext.getCellComparator()); - } else { - deleteFamilyBloomFilterWriter = null; - } - if (deleteFamilyBloomFilterWriter != null && LOG.isTraceEnabled()) { - LOG.trace("Delete Family Bloom filter type for " + path + ": " - + deleteFamilyBloomFilterWriter.getClass().getSimpleName()); + // initialize delete family Bloom filter when there is NO RowCol Bloom filter + if (this.bloomType != BloomType.ROWCOL) { + this.deleteFamilyBloomFilterWriter = BloomFilterFactory.createDeleteBloomAtWrite(conf, + cacheConf, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); + if (deleteFamilyBloomFilterWriter != null) { + deleteFamilyBloomContext = + new RowBloomContext(deleteFamilyBloomFilterWriter, fileContext.getCellComparator()); + } + } else { + this.deleteFamilyBloomFilterWriter = null; + } + if (deleteFamilyBloomFilterWriter != null && LOG.isTraceEnabled()) { + LOG.trace("Delete Family Bloom filter type for " + path + ": " + + deleteFamilyBloomFilterWriter.getClass().getSimpleName()); + } } } @@ -611,37 +713,11 @@ private void appendMetadata(final long maxSequenceId, final boolean majorCompact final Collection storeFiles) throws IOException { writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); - writer.appendFileInfo(COMPACTION_EVENT_KEY, toCompactionEventTrackerBytes(storeFiles)); + writer.appendFileInfo(COMPACTION_EVENT_KEY, + StoreFileWriter.buildCompactionEventTrackerBytes(this.compactedFilesSupplier, storeFiles)); appendTrackedTimestampsToMetadata(); } - /** - * Used when write {@link HStoreFile#COMPACTION_EVENT_KEY} to new file's file info. The - * compacted store files's name is needed. But if the compacted store file is a result of - * compaction, it's compacted files which still not archived is needed, too. And don't need to - * add compacted files recursively. If file A, B, C compacted to new file D, and file D - * compacted to new file E, will write A, B, C, D to file E's compacted files. So if file E - * compacted to new file F, will add E to F's compacted files first, then add E's compacted - * files: A, B, C, D to it. And no need to add D's compacted file, as D's compacted files has - * been in E's compacted files, too. See HBASE-20724 for more details. - * @param storeFiles The compacted store files to generate this new file - * @return bytes of CompactionEventTracker - */ - private byte[] toCompactionEventTrackerBytes(Collection storeFiles) { - Set notArchivedCompactedStoreFiles = this.compactedFilesSupplier.get().stream() - .map(sf -> sf.getPath().getName()).collect(Collectors.toSet()); - Set compactedStoreFiles = new HashSet<>(); - for (HStoreFile storeFile : storeFiles) { - compactedStoreFiles.add(storeFile.getFileInfo().getPath().getName()); - for (String csf : storeFile.getCompactedStoreFiles()) { - if (notArchivedCompactedStoreFiles.contains(csf)) { - compactedStoreFiles.add(csf); - } - } - } - return ProtobufUtil.toCompactionEventTrackerBytes(compactedStoreFiles); - } - /** * Writes meta data. Call before {@link #close()} since its written as meta data to this file. * @param maxSequenceId Maximum sequence id. @@ -760,6 +836,10 @@ private boolean closeGeneralBloomFilter() throws IOException { } private boolean closeDeleteFamilyBloomFilter() throws IOException { + if (multiTenantWriter) { + // Multi-tenant writer already attached delete-family blooms per section. + return false; + } boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter); // add the delete family Bloom filter writer @@ -794,6 +874,14 @@ private void appendFileInfo(byte[] key, byte[] value) throws IOException { writer.appendFileInfo(key, value); } + private void setCustomTieringTimeRangeSupplier(Supplier supplier) { + if (writer instanceof HFileWriterImpl) { + ((HFileWriterImpl) writer).setTimeRangeTrackerForTiering(supplier); + } else if (writer instanceof MultiTenantHFileWriter) { + ((MultiTenantHFileWriter) writer).setCustomTieringTimeRangeSupplier(supplier); + } + } + /** * For use in testing. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 069968294b84..f9a7d78e6a27 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -365,6 +365,11 @@ protected final List compact(final CompactionRequestImpl request, } writer = sinkFactory.createWriter(scanner, fd, dropCache, request.isMajor(), request.getWriterCreationTracker()); + // Thread compaction context to writer so downstream formats (e.g., v4 sections) inherit + if (writer instanceof org.apache.hadoop.hbase.regionserver.StoreFileWriter) { + ((org.apache.hadoop.hbase.regionserver.StoreFileWriter) writer) + .appendCompactionContext(request.isAllFiles(), request.getFiles()); + } finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId, throughputController, request, progress); if (!finished) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileIntegrationTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileIntegrationTest.java new file mode 100644 index 000000000000..883bca60a1a1 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileIntegrationTest.java @@ -0,0 +1,859 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Integration test for multi-tenant HFile functionality. + *

+ * This test validates the complete multi-tenant HFile workflow: + *

    + *
  1. Setup: Creates table with multi-tenant configuration
  2. + *
  3. Data Writing: Writes data for multiple tenants with distinct prefixes
  4. + *
  5. Flushing: Forces memstore flush to create multi-tenant HFile v4 files
  6. + *
  7. Verification: Tests various read patterns and tenant isolation
  8. + *
  9. Format Validation: Verifies HFile v4 structure and tenant sections
  10. + *
+ *

+ * The test ensures tenant data isolation, format compliance, and data integrity across different + * access patterns (GET, SCAN, tenant-specific scans). + */ +@Category(MediumTests.class) +public class MultiTenantHFileIntegrationTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(MultiTenantHFileIntegrationTest.class); + + private static final Logger LOG = LoggerFactory.getLogger(MultiTenantHFileIntegrationTest.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + private static final TableName TABLE_NAME = TableName.valueOf("TestMultiTenantTable"); + private static final byte[] FAMILY = Bytes.toBytes("f"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + + private static final int TENANT_PREFIX_LENGTH = 3; + private static final String[] TENANTS = + { "T01", "T02", "T03", "T04", "T05", "T06", "T07", "T08", "T09", "T10" }; + private static final int[] ROWS_PER_TENANT = { 5, 8, 12, 3, 15, 7, 20, 6, 10, 14 }; + private static final Map TENANT_DELETE_FAMILY_COUNTS = new HashMap<>(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + LOG.info("=== Setting up Multi-Tenant HFile Integration Test ==="); + Configuration conf = TEST_UTIL.getConfiguration(); + + // Configure multi-tenant HFile settings + conf.setInt(MultiTenantHFileWriter.TENANT_PREFIX_LENGTH, TENANT_PREFIX_LENGTH); + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + + LOG.info("Starting mini cluster with multi-tenant HFile configuration"); + LOG.info(" - Tenant prefix length: {}", TENANT_PREFIX_LENGTH); + LOG.info(" - HFile format version: {}", HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + + TEST_UTIL.startMiniCluster(1); + LOG.info("Mini cluster started successfully"); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + LOG.info("=== Shutting down Multi-Tenant HFile Integration Test ==="); + TEST_UTIL.shutdownMiniCluster(); + LOG.info("Mini cluster shutdown complete"); + } + + /** + * End-to-end test of multi-tenant HFile functionality. + *

+ * Test execution flow: + *

    + *
  1. Create table with multi-tenant configuration
  2. + *
  3. Write test data for {} tenants with varying row counts
  4. + *
  5. Flush memstore to create multi-tenant HFiles
  6. + *
  7. Verify data integrity using GET operations
  8. + *
  9. Verify data using full table SCAN
  10. + *
  11. Verify tenant isolation using tenant-specific scans
  12. + *
  13. Test edge cases and cross-tenant isolation
  14. + *
  15. Validate HFile format and tenant section structure
  16. + *
+ */ + @Test(timeout = 180000) + public void testMultiTenantHFileCreation() throws Exception { + LOG.info("=== Starting Multi-Tenant HFile Integration Test ==="); + LOG.info("Test will process {} tenants with {} total expected rows", TENANTS.length, + calculateTotalExpectedRows()); + + // Phase 1: Setup + LOG.info("Phase 1: Creating test table with multi-tenant configuration"); + createTestTable(); + + // Phase 2: Data Writing + LOG.info("Phase 2: Writing test data for {} tenants", TENANTS.length); + writeTestData(); + + // Phase 3: Pre-flush Verification + LOG.info("Phase 3: Verifying memstore state before flush"); + assertTableMemStoreNotEmpty(); + + // Phase 4: Flushing + LOG.info("Phase 4: Flushing memstore to create multi-tenant HFiles"); + flushTable(); + + // Phase 5: Post-flush Verification + LOG.info("Phase 5: Verifying memstore state after flush"); + assertTableMemStoreEmpty(); + + // Wait for HFiles to stabilize + LOG.info("Waiting for HFiles to stabilize..."); + Thread.sleep(2000); + + // Phase 6: Data Verification + LOG.info("Phase 6: Starting comprehensive data verification"); + verifyDataWithGet(); + verifyDataWithScan(); + verifyDataWithTenantSpecificScans(); + verifyEdgeCasesAndCrossTenantIsolation(); + + // Phase 7: HFile Format Verification + LOG.info("Phase 7: Verifying HFile format and structure"); + List hfilePaths = findHFilePaths(); + assertFalse("No HFiles found after flush", hfilePaths.isEmpty()); + LOG.info("Found {} HFiles for verification", hfilePaths.size()); + verifyHFileFormat(hfilePaths); + + LOG.info("=== Multi-tenant HFile integration test completed successfully ==="); + } + + /** + * Calculate total expected rows across all tenants. + * @return sum of rows across all tenants + */ + private static int calculateTotalExpectedRows() { + int total = 0; + for (int rows : ROWS_PER_TENANT) { + total += rows; + } + return total; + } + + /** + * Create test table with multi-tenant configuration. Sets up table properties required for + * multi-tenant HFile functionality. + */ + private void createTestTable() throws IOException { + try (Admin admin = TEST_UTIL.getAdmin()) { + TableDescriptorBuilder tableBuilder = TableDescriptorBuilder.newBuilder(TABLE_NAME); + + // Set multi-tenant properties + tableBuilder.setValue(MultiTenantHFileWriter.TABLE_TENANT_PREFIX_LENGTH, + String.valueOf(TENANT_PREFIX_LENGTH)); + tableBuilder.setValue(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED, "true"); + + tableBuilder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()); + + admin.createTable(tableBuilder.build()); + LOG.info("Created table {} with multi-tenant configuration", TABLE_NAME); + + try { + TEST_UTIL.waitTableAvailable(TABLE_NAME); + LOG.info("Table {} is now available", TABLE_NAME); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted waiting for table", e); + } + } + } + + /** + * Write test data for all tenants. Creates rows with format: {tenantId}row{paddedIndex} -> + * value_tenant-{tenantId}_row-{paddedIndex} + */ + private void writeTestData() throws IOException { + try (Connection connection = TEST_UTIL.getConnection(); + Table table = connection.getTable(TABLE_NAME)) { + + List batchPuts = new ArrayList<>(); + List batchDeletes = new ArrayList<>(); + TENANT_DELETE_FAMILY_COUNTS.clear(); + + LOG.info("Generating test data for {} tenants:", TENANTS.length); + for (int tenantIndex = 0; tenantIndex < TENANTS.length; tenantIndex++) { + String tenantId = TENANTS[tenantIndex]; + TENANT_DELETE_FAMILY_COUNTS.put(tenantId, 0); + int rowsForThisTenant = ROWS_PER_TENANT[tenantIndex]; + LOG.info(" - Tenant {}: {} rows", tenantId, rowsForThisTenant); + + for (int rowIndex = 0; rowIndex < rowsForThisTenant; rowIndex++) { + String rowKey = String.format("%srow%03d", tenantId, rowIndex); + Put putOperation = new Put(Bytes.toBytes(rowKey)); + + String cellValue = String.format("value_tenant-%s_row-%03d", tenantId, rowIndex); + putOperation.addColumn(FAMILY, QUALIFIER, Bytes.toBytes(cellValue)); + batchPuts.add(putOperation); + + if (rowIndex == 0 || rowIndex == rowsForThisTenant - 1) { + Delete delete = new Delete(Bytes.toBytes(rowKey)); + delete.addFamily(FAMILY, 0L); + batchDeletes.add(delete); + TENANT_DELETE_FAMILY_COUNTS.merge(tenantId, 1, Integer::sum); + } + } + } + + LOG.info("Writing {} total rows to table in batch operation", batchPuts.size()); + table.put(batchPuts); + if (!batchDeletes.isEmpty()) { + LOG.info("Writing {} delete family markers with timestamp 0", batchDeletes.size()); + table.delete(batchDeletes); + } + LOG.info("Successfully wrote all test data to table {}", TABLE_NAME); + } + } + + /** + * Verify that memstore contains data before flush. + */ + private void assertTableMemStoreNotEmpty() { + List regions = TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME); + long totalSize = regions.stream().mapToLong(HRegion::getMemStoreDataSize).sum(); + assertTrue("Table memstore should not be empty", totalSize > 0); + LOG.info("Memstore contains {} bytes of data before flush", totalSize); + } + + /** + * Verify that memstore is empty after flush. + */ + private void assertTableMemStoreEmpty() { + List regions = TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME); + long totalSize = regions.stream().mapToLong(HRegion::getMemStoreDataSize).sum(); + assertEquals("Table memstore should be empty after flush", 0, totalSize); + LOG.info("Memstore is empty after flush (size: {} bytes)", totalSize); + } + + /** + * Flush table to create HFiles on disk. + */ + private void flushTable() throws IOException { + LOG.info("Initiating flush operation for table {}", TABLE_NAME); + TEST_UTIL.flush(TABLE_NAME); + + // Wait for flush to complete + try { + Thread.sleep(5000); + TEST_UTIL.waitTableAvailable(TABLE_NAME, 30000); + LOG.info("Flush operation completed successfully"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.warn("Interrupted while waiting for flush", e); + } catch (Exception e) { + LOG.warn("Exception while waiting for table availability: {}", e.getMessage()); + } + } + + /** + * Verify data integrity using individual GET operations. Tests that each row can be retrieved + * correctly with expected values. + */ + private void verifyDataWithGet() throws Exception { + LOG.info("=== Verification Phase 1: GET Operations ==="); + + try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + Table table = conn.getTable(TABLE_NAME)) { + + int totalRowsVerified = 0; + + for (int tenantIndex = 0; tenantIndex < TENANTS.length; tenantIndex++) { + String tenant = TENANTS[tenantIndex]; + int rowsForThisTenant = ROWS_PER_TENANT[tenantIndex]; + + LOG.info("Verifying GET operations for tenant {}: {} rows", tenant, rowsForThisTenant); + + for (int i = 0; i < rowsForThisTenant; i++) { + String rowKey = tenant + "row" + String.format("%03d", i); + String expectedValue = "value_tenant-" + tenant + "_row-" + String.format("%03d", i); + + Get get = new Get(Bytes.toBytes(rowKey)); + get.addColumn(FAMILY, QUALIFIER); + + Result result = table.get(get); + if (result.isEmpty()) { + fail("No result found for row: " + rowKey); + } + + byte[] actualValue = result.getValue(FAMILY, QUALIFIER); + String actualValueStr = Bytes.toString(actualValue); + assertEquals("Value mismatch for row " + rowKey, expectedValue, actualValueStr); + totalRowsVerified++; + } + + LOG.info("Successfully verified {} GET operations for tenant {}", rowsForThisTenant, + tenant); + } + + LOG.info("GET verification completed: {}/{} rows verified successfully", totalRowsVerified, + calculateTotalExpectedRows()); + } + } + + /** + * Verify data integrity using full table SCAN. Tests complete data retrieval and checks for + * tenant data mixing. + */ + private void verifyDataWithScan() throws IOException { + LOG.info("=== Verification Phase 2: Full Table SCAN ==="); + + try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + Table table = connection.getTable(TABLE_NAME)) { + + org.apache.hadoop.hbase.client.Scan tableScan = new org.apache.hadoop.hbase.client.Scan(); + tableScan.addColumn(FAMILY, QUALIFIER); + + try ( + org.apache.hadoop.hbase.client.ResultScanner resultScanner = table.getScanner(tableScan)) { + int totalRowCount = 0; + + LOG.info("Starting full table scan to verify all data"); + + for (org.apache.hadoop.hbase.client.Result scanResult : resultScanner) { + String rowKey = Bytes.toString(scanResult.getRow()); + String extractedTenantId = rowKey.substring(0, TENANT_PREFIX_LENGTH); + + byte[] cellValue = scanResult.getValue(FAMILY, QUALIFIER); + if (cellValue != null) { + String actualValueString = Bytes.toString(cellValue); + if (!actualValueString.contains(extractedTenantId)) { + fail("Tenant data mixing detected: Row " + rowKey + " expected tenant " + + extractedTenantId + " but got value " + actualValueString); + } + } else { + fail("Missing value for row: " + rowKey); + } + + totalRowCount++; + } + + int expectedTotalRows = calculateTotalExpectedRows(); + assertEquals("Row count mismatch", expectedTotalRows, totalRowCount); + + LOG.info("Full table SCAN completed: {}/{} rows scanned successfully", totalRowCount, + expectedTotalRows); + } + } + } + + /** + * Verify tenant isolation using tenant-specific SCAN operations. Tests that each tenant's data + * can be accessed independently without cross-tenant leakage. + */ + private void verifyDataWithTenantSpecificScans() throws IOException { + LOG.info("=== Verification Phase 3: Tenant-Specific SCANs ==="); + + try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + Table table = connection.getTable(TABLE_NAME)) { + + int totalTenantsVerified = 0; + + for (int tenantIndex = 0; tenantIndex < TENANTS.length; tenantIndex++) { + String targetTenantId = TENANTS[tenantIndex]; + int expectedRowsForThisTenant = ROWS_PER_TENANT[tenantIndex]; + + LOG.info("Verifying tenant-specific scan for tenant {}: expecting {} rows", targetTenantId, + expectedRowsForThisTenant); + + org.apache.hadoop.hbase.client.Scan tenantScan = new org.apache.hadoop.hbase.client.Scan(); + tenantScan.addColumn(FAMILY, QUALIFIER); + tenantScan.withStartRow(Bytes.toBytes(targetTenantId + "row")); + tenantScan.withStopRow(Bytes.toBytes(targetTenantId + "row" + "\uFFFF")); + + try (org.apache.hadoop.hbase.client.ResultScanner tenantScanner = + table.getScanner(tenantScan)) { + int tenantRowCount = 0; + List foundRows = new ArrayList<>(); + + for (org.apache.hadoop.hbase.client.Result scanResult : tenantScanner) { + String rowKey = Bytes.toString(scanResult.getRow()); + foundRows.add(rowKey); + + if (!rowKey.startsWith(targetTenantId)) { + fail("Tenant scan violation: Found row " + rowKey + " in scan for tenant " + + targetTenantId); + } + + tenantRowCount++; + } + + // Debug logging to see which rows were found + LOG.info("Tenant {} scan found {} rows: {}", targetTenantId, tenantRowCount, foundRows); + + if (tenantRowCount != expectedRowsForThisTenant) { + // Generate expected rows for comparison + List expectedRows = new ArrayList<>(); + for (int j = 0; j < expectedRowsForThisTenant; j++) { + expectedRows.add(targetTenantId + "row" + String.format("%03d", j)); + } + LOG.error("Expected rows for {}: {}", targetTenantId, expectedRows); + LOG.error("Found rows for {}: {}", targetTenantId, foundRows); + + // Find missing rows + List missingRows = new ArrayList<>(expectedRows); + missingRows.removeAll(foundRows); + LOG.error("Missing rows for {}: {}", targetTenantId, missingRows); + } + + assertEquals("Row count mismatch for tenant " + targetTenantId, expectedRowsForThisTenant, + tenantRowCount); + + LOG.info("Tenant {} scan successful: {}/{} rows verified", targetTenantId, tenantRowCount, + expectedRowsForThisTenant); + } + + totalTenantsVerified++; + } + + LOG.info("Tenant-specific SCAN verification completed: {}/{} tenants verified successfully", + totalTenantsVerified, TENANTS.length); + } + } + + /** + * Verify edge cases and cross-tenant isolation boundaries. Tests non-existent tenant queries, + * empty scan behavior, and tenant boundary conditions. + */ + private void verifyEdgeCasesAndCrossTenantIsolation() throws IOException { + LOG.info("=== Verification Phase 4: Edge Cases and Cross-Tenant Isolation ==="); + + try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + Table table = conn.getTable(TABLE_NAME)) { + + // Test 1: Non-existent tenant prefix + LOG.info("Test 1: Scanning with non-existent tenant prefix"); + verifyNonExistentTenantScan(table); + + // Test 2: Tenant boundary isolation + LOG.info("Test 2: Verifying tenant boundary isolation"); + verifyTenantBoundaries(table); + + // Test 3: Empty scan returns all rows + LOG.info("Test 3: Verifying empty scan behavior"); + verifyEmptyScan(table); + + LOG.info("Edge cases and cross-tenant isolation verification completed successfully"); + } + } + + /** + * Verify that scanning with a non-existent tenant prefix returns no results. + */ + private void verifyNonExistentTenantScan(Table table) throws IOException { + String nonExistentPrefix = "ZZZ"; + LOG.info("Testing scan with non-existent tenant prefix: {}", nonExistentPrefix); + + org.apache.hadoop.hbase.client.Scan scan = new org.apache.hadoop.hbase.client.Scan(); + scan.addColumn(FAMILY, QUALIFIER); + scan.withStartRow(Bytes.toBytes(nonExistentPrefix + "row")); + scan.withStopRow(Bytes.toBytes(nonExistentPrefix + "row" + "\uFFFF")); + + try (org.apache.hadoop.hbase.client.ResultScanner scanner = table.getScanner(scan)) { + int rowCount = 0; + for (org.apache.hadoop.hbase.client.Result result : scanner) { + LOG.error("Unexpected row found for non-existent tenant: {}", + Bytes.toString(result.getRow())); + rowCount++; + } + + assertEquals("Scan with non-existent tenant prefix should return no results", 0, rowCount); + LOG.info("Non-existent tenant scan test passed: {} rows returned", rowCount); + } + } + + /** + * Verify tenant boundaries are properly enforced by scanning across adjacent tenant boundaries. + * This test scans from the last row of one tenant to the first row of the next tenant to ensure + * proper tenant isolation at boundaries. + */ + private void verifyTenantBoundaries(Table table) throws IOException { + LOG.info("Verifying tenant boundary isolation between adjacent tenants"); + + int boundariesTested = 0; + + // Test boundaries between adjacent tenants + for (int i = 0; i < TENANTS.length - 1; i++) { + String tenant1 = TENANTS[i]; + String tenant2 = TENANTS[i + 1]; + int tenant1RowCount = ROWS_PER_TENANT[i]; + int tenant2RowCount = ROWS_PER_TENANT[i + 1]; + + LOG.info("Testing boundary between tenant {} ({} rows) and tenant {} ({} rows)", tenant1, + tenant1RowCount, tenant2, tenant2RowCount); + + // Create a scan that covers the boundary between tenant1 and tenant2 + org.apache.hadoop.hbase.client.Scan scan = new org.apache.hadoop.hbase.client.Scan(); + scan.addColumn(FAMILY, QUALIFIER); + + // Set start row to last row of tenant1 + String startRow = tenant1 + "row" + String.format("%03d", tenant1RowCount - 1); + // Set stop row to second row of tenant2 (to ensure we get at least first row of tenant2) + String stopRow = tenant2 + "row" + String.format("%03d", Math.min(1, tenant2RowCount - 1)); + + LOG.info(" Boundary scan range: [{}] to [{}]", startRow, stopRow); + + scan.withStartRow(Bytes.toBytes(startRow)); + scan.withStopRow(Bytes.toBytes(stopRow)); + + try (org.apache.hadoop.hbase.client.ResultScanner scanner = table.getScanner(scan)) { + int tenant1Count = 0; + int tenant2Count = 0; + List scannedRows = new ArrayList<>(); + + for (org.apache.hadoop.hbase.client.Result result : scanner) { + String rowKey = Bytes.toString(result.getRow()); + scannedRows.add(rowKey); + + if (rowKey.startsWith(tenant1)) { + tenant1Count++; + } else if (rowKey.startsWith(tenant2)) { + tenant2Count++; + } else { + LOG.error("Unexpected tenant in boundary scan: {}", rowKey); + fail("Unexpected tenant in boundary scan: " + rowKey); + } + } + + LOG.info(" Boundary scan results:"); + LOG.info(" - Rows from {}: {}", tenant1, tenant1Count); + LOG.info(" - Rows from {}: {}", tenant2, tenant2Count); + LOG.info(" - Total rows scanned: {}", scannedRows.size()); + + // Log the actual rows found for debugging + if (scannedRows.size() <= 5) { + LOG.info(" - Scanned rows: {}", scannedRows); + } else { + LOG.info(" - Scanned rows (first 5): {}", scannedRows.subList(0, 5)); + } + + // We should find the last row from tenant1 + assertTrue("Should find at least one row from tenant " + tenant1, tenant1Count > 0); + + // We should find at least the first row from tenant2 (if tenant2 has any rows) + if (tenant2RowCount > 0) { + assertTrue("Should find at least one row from tenant " + tenant2, tenant2Count > 0); + } + + // Ensure proper tenant separation - no unexpected tenants + int totalFoundRows = tenant1Count + tenant2Count; + assertEquals("All scanned rows should belong to expected tenants", scannedRows.size(), + totalFoundRows); + + LOG.info(" Boundary test passed for tenants {} and {}", tenant1, tenant2); + } + + boundariesTested++; + } + + LOG.info("Tenant boundary verification completed: {}/{} boundaries tested successfully", + boundariesTested, TENANTS.length - 1); + } + + /** + * Verify that an empty scan returns all rows across all tenants. + */ + private void verifyEmptyScan(Table table) throws IOException { + LOG.info("Testing empty scan to verify it returns all rows across all tenants"); + + org.apache.hadoop.hbase.client.Scan emptyScan = new org.apache.hadoop.hbase.client.Scan(); + emptyScan.addColumn(FAMILY, QUALIFIER); + + try (org.apache.hadoop.hbase.client.ResultScanner emptyScanner = table.getScanner(emptyScan)) { + int rowCount = 0; + for (org.apache.hadoop.hbase.client.Result result : emptyScanner) { + rowCount++; + } + + int expectedTotal = calculateTotalExpectedRows(); + assertEquals("Empty scan should return all rows", expectedTotal, rowCount); + LOG.info("Empty scan test passed: {}/{} rows returned", rowCount, expectedTotal); + } + } + + /** + * Verify HFile format and multi-tenant structure. Validates that HFiles are properly formatted as + * v4 with tenant sections. + */ + private void verifyHFileFormat(List hfilePaths) throws IOException { + LOG.info("=== HFile Format Verification ==="); + LOG.info("Verifying {} HFiles for multi-tenant format compliance", hfilePaths.size()); + + FileSystem fs = TEST_UTIL.getTestFileSystem(); + Configuration conf = TEST_UTIL.getConfiguration(); + CacheConfig cacheConf = new CacheConfig(conf); + + int totalHFilesVerified = 0; + int totalCellsFoundAcrossAllFiles = 0; + + for (Path path : hfilePaths) { + LOG.info("Verifying HFile: {}", path.getName()); + + try (HFile.Reader reader = HFile.createReader(fs, path, cacheConf, true, conf)) { + // Verify HFile version + int version = reader.getTrailer().getMajorVersion(); + assertEquals("HFile should be version 4", HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT, + version); + LOG.info(" HFile version: {} (correct)", version); + + // Verify reader type + assertTrue("Reader should be an AbstractMultiTenantReader", + reader instanceof AbstractMultiTenantReader); + LOG.info(" Reader type: AbstractMultiTenantReader (correct)"); + + AbstractMultiTenantReader mtReader = (AbstractMultiTenantReader) reader; + byte[][] allTenantSectionIds = mtReader.getAllTenantSectionIds(); + + assertTrue("Should have tenant sections", allTenantSectionIds.length > 0); + LOG.info(" Found {} tenant sections in HFile", allTenantSectionIds.length); + + int totalCellsInThisFile = 0; + int sectionsWithData = 0; + + for (byte[] tenantSectionId : allTenantSectionIds) { + String tenantId = Bytes.toString(tenantSectionId); + try { + java.lang.reflect.Method getSectionReaderMethod = + AbstractMultiTenantReader.class.getDeclaredMethod("getSectionReader", byte[].class); + getSectionReaderMethod.setAccessible(true); + Object sectionReaderLeaseObj = getSectionReaderMethod.invoke(mtReader, tenantSectionId); + + if (sectionReaderLeaseObj != null) { + try (AbstractMultiTenantReader.SectionReaderLease lease = + (AbstractMultiTenantReader.SectionReaderLease) sectionReaderLeaseObj) { + HFileReaderImpl sectionHFileReader = lease.getReader(); + + HFileInfo sectionInfo = sectionHFileReader.getHFileInfo(); + byte[] deleteCountBytes = sectionInfo + .get(org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT); + if (deleteCountBytes != null) { + long deleteCount = Bytes.toLong(deleteCountBytes); + int expectedCount = TENANT_DELETE_FAMILY_COUNTS.getOrDefault(tenantId, 0); + assertEquals("Delete family count mismatch for tenant " + tenantId, expectedCount, + deleteCount); + } + + HFileScanner sectionScanner = sectionHFileReader.getScanner(conf, false, false); + + boolean hasData = sectionScanner.seekTo(); + if (hasData) { + int sectionCellCount = 0; + do { + Cell cell = sectionScanner.getCell(); + if (cell != null) { + // Verify tenant prefix matches section ID for every entry + byte[] rowKeyBytes = CellUtil.cloneRow(cell); + byte[] rowTenantPrefix = new byte[TENANT_PREFIX_LENGTH]; + System.arraycopy(rowKeyBytes, 0, rowTenantPrefix, 0, TENANT_PREFIX_LENGTH); + + assertTrue("Row tenant prefix should match section ID", + Bytes.equals(tenantSectionId, rowTenantPrefix)); + + if (cell.getType() == Cell.Type.Put) { + sectionCellCount++; + totalCellsInThisFile++; + } + } + } while (sectionScanner.next()); + + assertTrue("Should have found data in tenant section", sectionCellCount > 0); + sectionsWithData++; + LOG.info(" Section {}: {} cells", tenantId, sectionCellCount); + } + } + } + } catch (Exception e) { + LOG.warn("Failed to access tenant section: " + tenantId, e); + } + } + + LOG.info(" Tenant sections with data: {}/{}", sectionsWithData, + allTenantSectionIds.length); + LOG.info(" Total cells in this HFile: {}", totalCellsInThisFile); + totalCellsFoundAcrossAllFiles += totalCellsInThisFile; + + // Verify HFile metadata contains multi-tenant information + LOG.info(" Verifying HFile metadata and structure"); + verifyHFileMetadata(reader, allTenantSectionIds, mtReader); + + LOG.info(" HFile verification completed for: {}", path.getName()); + totalHFilesVerified++; + } + } + + int expectedTotal = calculateTotalExpectedRows(); + assertEquals("Should have found all cells across all HFiles", expectedTotal, + totalCellsFoundAcrossAllFiles); + + LOG.info("HFile format verification completed successfully:"); + LOG.info(" - HFiles verified: {}/{}", totalHFilesVerified, hfilePaths.size()); + LOG.info(" - Total cells verified: {}/{}", totalCellsFoundAcrossAllFiles, expectedTotal); + LOG.info(" - All HFiles are properly formatted as multi-tenant v4"); + } + + /** + * Verify HFile metadata contains expected multi-tenant information. Checks for section count, + * tenant index levels, and other v4 metadata. + */ + private void verifyHFileMetadata(HFile.Reader reader, byte[][] allTenantSectionIds, + AbstractMultiTenantReader mtReader) throws IOException { + HFileInfo fileInfo = reader.getHFileInfo(); + if (fileInfo == null) { + LOG.warn(" - HFile info is null - cannot verify metadata"); + return; + } + + // Verify section count metadata + byte[] sectionCountBytes = + fileInfo.get(Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_SECTION_COUNT)); + if (sectionCountBytes != null) { + int sectionCount = Bytes.toInt(sectionCountBytes); + LOG.info(" - HFile section count: {}", sectionCount); + assertTrue("HFile should have tenant sections", sectionCount > 0); + assertEquals("Section count should match found tenant sections", allTenantSectionIds.length, + sectionCount); + } else { + LOG.warn(" - Missing SECTION_COUNT metadata in HFile info"); + } + + // Verify tenant index structure metadata + byte[] tenantIndexLevelsBytes = + fileInfo.get(Bytes.toBytes(MultiTenantHFileWriter.FILEINFO_TENANT_INDEX_LEVELS)); + if (tenantIndexLevelsBytes != null) { + int tenantIndexLevels = Bytes.toInt(tenantIndexLevelsBytes); + LOG.info(" - Tenant index levels: {}", tenantIndexLevels); + assertTrue("HFile should have tenant index levels", tenantIndexLevels > 0); + + // Log index structure details + if (tenantIndexLevels == 1) { + LOG.info(" - Using single-level tenant index (suitable for {} sections)", + allTenantSectionIds.length); + } else { + LOG.info(" - Using multi-level tenant index ({} levels for {} sections)", + tenantIndexLevels, allTenantSectionIds.length); + } + } else { + LOG.warn(" - Missing TENANT_INDEX_LEVELS metadata in HFile info"); + } + + // Verify reader provides multi-tenant specific information + LOG.info(" - Multi-tenant reader statistics:"); + LOG.info(" * Total sections: {}", mtReader.getTotalSectionCount()); + LOG.info(" * Tenant index levels: {}", mtReader.getTenantIndexLevels()); + LOG.info(" * Tenant index max chunk size: {}", mtReader.getTenantIndexMaxChunkSize()); + + // Verify consistency between metadata and reader state + if (tenantIndexLevelsBytes != null) { + int metadataTenantIndexLevels = Bytes.toInt(tenantIndexLevelsBytes); + assertEquals("Tenant index levels should match between metadata and reader", + metadataTenantIndexLevels, mtReader.getTenantIndexLevels()); + } + + assertEquals("Total section count should match found sections", allTenantSectionIds.length, + mtReader.getTotalSectionCount()); + + LOG.info(" - HFile metadata verification passed"); + } + + /** + * Find all HFiles created for the test table. Scans the filesystem to locate HFiles in the + * table's directory structure. + */ + private List findHFilePaths() throws IOException { + LOG.info("Searching for HFiles in table directory structure"); + + List hfilePaths = new ArrayList<>(); + + FileSystem fs = TEST_UTIL.getTestFileSystem(); + Path rootDir = TEST_UTIL.getDataTestDirOnTestFS(); + Path tableDir = new Path(rootDir, "data/default/" + TABLE_NAME.getNameAsString()); + + if (fs.exists(tableDir)) { + FileStatus[] regionDirs = fs.listStatus(tableDir); + LOG.info("Found {} region directories to scan", regionDirs.length); + + for (FileStatus regionDir : regionDirs) { + if (regionDir.isDirectory() && !regionDir.getPath().getName().startsWith(".")) { + Path familyDir = new Path(regionDir.getPath(), Bytes.toString(FAMILY)); + + if (fs.exists(familyDir)) { + FileStatus[] hfiles = fs.listStatus(familyDir); + + for (FileStatus hfile : hfiles) { + if ( + !hfile.getPath().getName().startsWith(".") + && !hfile.getPath().getName().endsWith(".tmp") + ) { + hfilePaths.add(hfile.getPath()); + LOG.info("Found HFile: {} (size: {} bytes)", hfile.getPath().getName(), + hfile.getLen()); + } + } + } + } + } + } + + LOG.info("HFile discovery completed: {} HFiles found", hfilePaths.size()); + return hfilePaths; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileMultiLevelIndexTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileMultiLevelIndexTest.java new file mode 100644 index 000000000000..ef5bf79f0fe7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileMultiLevelIndexTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Validates that multi-tenant HFile v4 builds and reads a multi-level section index. This test + * forces a multi-level index by setting small chunk sizes and writing many tenants. + */ +@Category(MediumTests.class) +public class MultiTenantHFileMultiLevelIndexTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(MultiTenantHFileMultiLevelIndexTest.class); + + private static final Logger LOG = + LoggerFactory.getLogger(MultiTenantHFileMultiLevelIndexTest.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + private static final TableName TABLE_NAME = TableName.valueOf("TestMultiLevelIndex"); + private static final byte[] FAMILY = Bytes.toBytes("f"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + + private static final int TENANT_PREFIX_LENGTH = 3; + // Force small chunking so we create multiple leaf blocks and an intermediate level. + private static final int FORCED_MAX_CHUNK_SIZE = 3; // entries per index block + private static final int FORCED_MIN_INDEX_ENTRIES = 4; // root fanout threshold + + // Create enough tenants to exceed FORCED_MAX_CHUNK_SIZE and FORCED_MIN_INDEX_ENTRIES + private static final int NUM_TENANTS = 20; + private static final int ROWS_PER_TENANT = 2; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + conf.setInt(MultiTenantHFileWriter.TENANT_PREFIX_LENGTH, TENANT_PREFIX_LENGTH); + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + // Force multi-level section index with small blocks + conf.setInt(SectionIndexManager.SECTION_INDEX_MAX_CHUNK_SIZE, FORCED_MAX_CHUNK_SIZE); + conf.setInt(SectionIndexManager.SECTION_INDEX_MIN_NUM_ENTRIES, FORCED_MIN_INDEX_ENTRIES); + TEST_UTIL.startMiniCluster(1); + try (Admin admin = TEST_UTIL.getAdmin()) { + TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(TABLE_NAME); + tdb.setValue(MultiTenantHFileWriter.TABLE_TENANT_PREFIX_LENGTH, + String.valueOf(TENANT_PREFIX_LENGTH)); + tdb.setValue(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED, "true"); + tdb.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()); + admin.createTable(tdb.build()); + } + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test(timeout = 120000) + public void testMultiLevelSectionIndexTraversal() throws Exception { + writeManyTenants(); + TEST_UTIL.flush(TABLE_NAME); + + // Wait a bit for files to land + Thread.sleep(2000); + + List hfiles = findHFiles(); + assertTrue("Expected at least one HFile", !hfiles.isEmpty()); + + int totalRows = 0; + java.util.Set uniqueSectionIds = new java.util.HashSet<>(); + for (Path p : hfiles) { + try (HFile.Reader r = HFile.createReader(TEST_UTIL.getTestFileSystem(), p, + new CacheConfig(TEST_UTIL.getConfiguration()), true, TEST_UTIL.getConfiguration())) { + assertEquals("HFile should be version 4", HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT, + r.getTrailer().getMajorVersion()); + assertTrue("Reader should be multi-tenant", r instanceof AbstractMultiTenantReader); + + // Validate multi-level index per trailer + int levels = r.getTrailer().getNumDataIndexLevels(); + LOG.info("HFile {} trailer reports section index levels: {}", p.getName(), levels); + assertTrue("Expected multi-level section index (>=2 levels)", levels >= 2); + + byte[][] tenantSections = ((AbstractMultiTenantReader) r).getAllTenantSectionIds(); + for (byte[] id : tenantSections) { + uniqueSectionIds.add(Bytes.toStringBinary(id)); + } + + // Scan all data via the multi-tenant reader to ensure traversal works across levels + HFileScanner scanner = + ((AbstractMultiTenantReader) r).getScanner(TEST_UTIL.getConfiguration(), false, false); + int rowsInThisFile = 0; + if (scanner.seekTo()) { + do { + rowsInThisFile++; + } while (scanner.next()); + } + LOG.info("HFile {} contains {} cells", p.getName(), rowsInThisFile); + totalRows += rowsInThisFile; + } + } + + assertEquals("Unique tenant sections across all files should equal tenants", NUM_TENANTS, + uniqueSectionIds.size()); + + assertEquals("Total cells should match expected write count", NUM_TENANTS * ROWS_PER_TENANT, + totalRows); + } + + private static void writeManyTenants() throws IOException { + try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + Table table = conn.getTable(TABLE_NAME)) { + List puts = new ArrayList<>(); + for (int t = 0; t < NUM_TENANTS; t++) { + String tenant = String.format("T%02d", t); // e.g., T00, T01 ... T19 + String tenantPrefix = tenant; // length 3 when combined with another char? Ensure 3 chars + // Ensure prefix is exactly TENANT_PREFIX_LENGTH by padding if needed + if (tenantPrefix.length() < TENANT_PREFIX_LENGTH) { + tenantPrefix = String.format("%-3s", tenantPrefix).replace(' ', 'X'); + } else if (tenantPrefix.length() > TENANT_PREFIX_LENGTH) { + tenantPrefix = tenantPrefix.substring(0, TENANT_PREFIX_LENGTH); + } + for (int i = 0; i < ROWS_PER_TENANT; i++) { + String row = tenantPrefix + "row" + String.format("%03d", i); + Put p = new Put(Bytes.toBytes(row)); + p.addColumn(FAMILY, QUALIFIER, Bytes.toBytes("v-" + tenantPrefix + "-" + i)); + puts.add(p); + } + } + table.put(puts); + } + } + + private static List findHFiles() throws IOException { + List hfiles = new ArrayList<>(); + FileSystem fs = TEST_UTIL.getTestFileSystem(); + Path rootDir = TEST_UTIL.getDataTestDirOnTestFS(); + Path tableDir = new Path(rootDir, "data/default/" + TABLE_NAME.getNameAsString()); + if (!fs.exists(tableDir)) { + return hfiles; + } + for (FileStatus regionDir : fs.listStatus(tableDir)) { + if (!regionDir.isDirectory() || regionDir.getPath().getName().startsWith(".")) { + continue; + } + Path familyDir = new Path(regionDir.getPath(), Bytes.toString(FAMILY)); + if (!fs.exists(familyDir)) { + continue; + } + for (FileStatus hfile : fs.listStatus(familyDir)) { + if ( + !hfile.getPath().getName().startsWith(".") && !hfile.getPath().getName().endsWith(".tmp") + ) { + hfiles.add(hfile.getPath()); + } + } + } + return hfiles; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileSplittingTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileSplittingTest.java new file mode 100644 index 000000000000..6189bf4d8f63 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/MultiTenantHFileSplittingTest.java @@ -0,0 +1,1008 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Integration test for HFile v4 multi-tenant splitting logic using isolated test pattern. + *

+ * Each test method runs independently with its own fresh cluster to ensure complete isolation and + * avoid connection interference issues between tests. + *

+ * This test validates the complete multi-tenant HFile v4 splitting workflow: + *

    + *
  1. Setup: Creates table with multi-tenant configuration
  2. + *
  3. Data Writing: Writes large datasets with different tenant distributions
  4. + *
  5. Flushing: Forces memstore flush to create multi-tenant HFile v4 files
  6. + *
  7. Splitting: Tests midkey calculation and file splitting
  8. + *
  9. Verification: Validates split balance and data integrity
  10. + *
+ */ +@Category(MediumTests.class) +public class MultiTenantHFileSplittingTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(MultiTenantHFileSplittingTest.class); + + private static final Logger LOG = LoggerFactory.getLogger(MultiTenantHFileSplittingTest.class); + + private HBaseTestingUtil testUtil; + + private static final byte[] FAMILY = Bytes.toBytes("f"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + private static final int TENANT_PREFIX_LENGTH = 3; + + @Before + public void setUp() throws Exception { + LOG.info("=== Setting up isolated test environment ==="); + + // Create fresh testing utility for each test + testUtil = new HBaseTestingUtil(); + + // Configure test settings + Configuration conf = testUtil.getConfiguration(); + + // Set HFile format version for multi-tenant support + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + conf.setInt(MultiTenantHFileWriter.TENANT_PREFIX_LENGTH, TENANT_PREFIX_LENGTH); + + // Set smaller region size to make splits easier to trigger + conf.setLong("hbase.hregion.max.filesize", 10 * 1024 * 1024); // 10MB + conf.setInt("hbase.regionserver.region.split.policy.check.period", 1000); + + // Use policy that allows manual splits + conf.set("hbase.regionserver.region.split.policy", + "org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy"); + + // Configure mini cluster settings for stability + conf.setInt("hbase.regionserver.msginterval", 100); + conf.setInt("hbase.client.pause", 250); + conf.setInt("hbase.client.retries.number", 6); + conf.setBoolean("hbase.master.enabletable.roundrobin", true); + + // Increase timeouts for split operations + conf.setLong("hbase.regionserver.fileSplitTimeout", 600000); // 10 minutes + conf.setInt("hbase.client.operation.timeout", 600000); // 10 minutes + + LOG.info("Configured HFile format version: {}", conf.getInt(HFile.FORMAT_VERSION_KEY, -1)); + + // Start fresh mini cluster for this test + LOG.info("Starting fresh mini cluster for test"); + testUtil.startMiniCluster(1); + + // Wait for cluster to be ready + testUtil.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + + LOG.info("Fresh cluster ready for test"); + } + + @After + public void tearDown() throws Exception { + LOG.info("=== Cleaning up isolated test environment ==="); + + if (testUtil != null) { + try { + testUtil.shutdownMiniCluster(); + LOG.info("Successfully shut down test cluster"); + } catch (Exception e) { + LOG.warn("Error during cluster shutdown", e); + } + } + } + + /** + * Test 1: Single tenant with large amount of data + */ + @Test(timeout = 600000) // 10 minute timeout + public void testSingleTenantSplitting() throws Exception { + String[] tenants = { "T01" }; + int[] rowsPerTenant = { 10000 }; + + executeTestScenario(tenants, rowsPerTenant); + } + + /** + * Test 2: Three tenants with even distribution + */ + @Test(timeout = 600000) // 10 minute timeout + public void testEvenDistributionSplitting() throws Exception { + String[] tenants = { "T01", "T02", "T03" }; + int[] rowsPerTenant = { 3000, 3000, 3000 }; + + executeTestScenario(tenants, rowsPerTenant); + } + + /** + * Test 3: Three tenants with uneven distribution + */ + @Test(timeout = 600000) // 10 minute timeout + public void testUnevenDistributionSplitting() throws Exception { + String[] tenants = { "T01", "T02", "T03" }; + int[] rowsPerTenant = { 1000, 2000, 1000 }; + + executeTestScenario(tenants, rowsPerTenant); + } + + /** + * Test 4: Skewed distribution with one dominant tenant + */ + @Test(timeout = 600000) // 10 minute timeout + public void testSkewedDistributionSplitting() throws Exception { + String[] tenants = { "T01", "T02", "T03", "T04" }; + int[] rowsPerTenant = { 100, 100, 5000, 100 }; + + executeTestScenario(tenants, rowsPerTenant); + } + + /** + * Test 5: Many small tenants + */ + @Test(timeout = 600000) // 10 minute timeout + public void testManySmallTenantsSplitting() throws Exception { + String[] tenants = { "T01", "T02", "T03", "T04", "T05", "T06", "T07", "T08", "T09", "T10" }; + int[] rowsPerTenant = { 500, 500, 500, 500, 500, 500, 500, 500, 500, 500 }; + + executeTestScenario(tenants, rowsPerTenant); + } + + /** + * Test 6: Few large tenants + */ + @Test(timeout = 600000) // 10 minute timeout + public void testFewLargeTenantsSplitting() throws Exception { + String[] tenants = { "T01", "T02" }; + int[] rowsPerTenant = { 5000, 5000 }; + + executeTestScenario(tenants, rowsPerTenant); + } + + /** + * Execute a test scenario with the given configuration. The table will be created fresh for this + * test. + */ + private void executeTestScenario(String[] tenants, int[] rowsPerTenant) throws Exception { + LOG.info("=== Starting test scenario ==="); + + // Generate unique table name for this test + String testName = Thread.currentThread().getStackTrace()[2].getMethodName(); + TableName tableName = TableName.valueOf(testName + "_" + System.currentTimeMillis()); + + // Validate input parameters + if (tenants.length != rowsPerTenant.length) { + throw new IllegalArgumentException("Tenants and rowsPerTenant arrays must have same length"); + } + + try { + // Phase 1: Create fresh table + LOG.info("Phase 1: Creating fresh table {}", tableName); + createTestTable(tableName); + + // Wait for table to be ready + Thread.sleep(1000); + + // Phase 2: Write test data + LOG.info("Phase 2: Writing test data"); + writeTestData(tableName, tenants, rowsPerTenant); + + // Phase 3: Flush memstore to create HFiles + LOG.info("Phase 3: Flushing table"); + testUtil.flush(tableName); + + // Wait for flush to complete + Thread.sleep(2000); + + // Phase 4: Verify midkey before split + LOG.info("Phase 4: Verifying midkey calculation"); + verifyMidkeyCalculation(tableName, tenants, rowsPerTenant); + + // Phase 5: Trigger split + LOG.info("Phase 5: Triggering region split"); + triggerRegionSplit(tenants, rowsPerTenant, tableName); + + // Phase 6: Compact after split to ensure proper HFile structure + LOG.info("Phase 6: Compacting table after split"); + testUtil.compact(tableName, true); // Major compaction + + // Wait for compaction to complete + Thread.sleep(3000); + + // Phase 7: Comprehensive data integrity verification after split + LOG.info("Phase 7: Starting comprehensive data integrity verification after split"); + verifyDataIntegrityWithScanning(tableName, tenants, rowsPerTenant); + verifyDataIntegrityAfterSplit(tableName, tenants, rowsPerTenant); + + LOG.info("=== Test scenario completed successfully ==="); + + } catch (Exception e) { + LOG.error("Test scenario failed", e); + throw e; + } finally { + // Clean up table + try { + if (testUtil.getAdmin() != null && testUtil.getAdmin().tableExists(tableName)) { + testUtil.deleteTable(tableName); + } + } catch (Exception cleanupException) { + LOG.warn("Failed to cleanup table {}: {}", tableName, cleanupException.getMessage()); + } + } + } + + /** + * Create test table with multi-tenant configuration. + */ + private void createTestTable(TableName tableName) throws IOException, InterruptedException { + LOG.info("Creating table: {} with multi-tenant configuration", tableName); + + // Build table descriptor with multi-tenant properties + TableDescriptorBuilder tableBuilder = TableDescriptorBuilder.newBuilder(tableName); + + // Set multi-tenant properties + tableBuilder.setValue(MultiTenantHFileWriter.TABLE_TENANT_PREFIX_LENGTH, + String.valueOf(TENANT_PREFIX_LENGTH)); + tableBuilder.setValue(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED, "true"); + + // Configure column family with proper settings + ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY); + + // Ensure HFile v4 format is used at column family level + cfBuilder.setValue(HFile.FORMAT_VERSION_KEY, + String.valueOf(HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT)); + + // Set smaller block size for easier testing + cfBuilder.setBlocksize(8 * 1024); // 8KB blocks + + tableBuilder.setColumnFamily(cfBuilder.build()); + + // Create the table + testUtil.createTable(tableBuilder.build(), null); + + LOG.info("Created table {} with multi-tenant configuration", tableName); + } + + /** + * Write test data for all tenants in lexicographic order to avoid key ordering violations. + */ + private void writeTestData(TableName tableName, String[] tenants, int[] rowsPerTenant) + throws IOException { + try (Connection connection = ConnectionFactory.createConnection(testUtil.getConfiguration()); + Table table = connection.getTable(tableName)) { + + List batchPuts = new ArrayList<>(); + + // Generate all row keys first + for (int tenantIndex = 0; tenantIndex < tenants.length; tenantIndex++) { + String tenantId = tenants[tenantIndex]; + int rowsForThisTenant = rowsPerTenant[tenantIndex]; + + for (int rowIndex = 0; rowIndex < rowsForThisTenant; rowIndex++) { + String rowKey = String.format("%srow%05d", tenantId, rowIndex); + Put putOperation = new Put(Bytes.toBytes(rowKey)); + + String cellValue = String.format("value_tenant-%s_row-%05d", tenantId, rowIndex); + putOperation.addColumn(FAMILY, QUALIFIER, Bytes.toBytes(cellValue)); + batchPuts.add(putOperation); + } + } + + // Sort puts by row key to ensure lexicographic ordering + batchPuts.sort((p1, p2) -> Bytes.compareTo(p1.getRow(), p2.getRow())); + + LOG.info("Writing {} total rows to table in lexicographic order", batchPuts.size()); + table.put(batchPuts); + LOG.info("Successfully wrote all test data to table {}", tableName); + } + } + + /** + * Verify midkey calculation for the HFile. + */ + private void verifyMidkeyCalculation(TableName tableName, String[] tenants, int[] rowsPerTenant) + throws IOException { + LOG.info("Verifying midkey calculation for table: {}", tableName); + + // Find the HFile path for the table + List hfilePaths = findHFilePaths(tableName); + assertTrue("Should have at least one HFile", hfilePaths.size() > 0); + + Path hfilePath = hfilePaths.get(0); // Use the first HFile + LOG.info("Checking midkey for HFile: {}", hfilePath); + + FileSystem fs = testUtil.getTestFileSystem(); + CacheConfig cacheConf = new CacheConfig(testUtil.getConfiguration()); + + try (HFile.Reader reader = + HFile.createReader(fs, hfilePath, cacheConf, true, testUtil.getConfiguration())) { + assertTrue("Reader should be AbstractMultiTenantReader", + reader instanceof AbstractMultiTenantReader); + + // Get the midkey + Optional midkey = reader.midKey(); + assertTrue("Midkey should be present", midkey.isPresent()); + + String midkeyString = Bytes.toString(CellUtil.cloneRow(midkey.get())); + LOG.info("Midkey: {}", midkeyString); + + // Analyze the midkey + int totalRows = 0; + for (int rows : rowsPerTenant) { + totalRows += rows; + } + + // Log HFile properties + LOG.info("HFile properties:"); + LOG.info(" - First key: {}", + reader.getFirstRowKey().isPresent() + ? Bytes.toString(reader.getFirstRowKey().get()) + : "N/A"); + LOG.info(" - Last key: {}", + reader.getLastRowKey().isPresent() ? Bytes.toString(reader.getLastRowKey().get()) : "N/A"); + LOG.info(" - Entry count: {}", reader.getEntries()); + + // Determine which tenant and position within that tenant + String midkeyTenant = midkeyString.substring(0, TENANT_PREFIX_LENGTH); + int midkeyPosition = 0; + boolean foundTenant = false; + + for (int i = 0; i < tenants.length; i++) { + if (tenants[i].equals(midkeyTenant)) { + int rowNum = Integer.parseInt(midkeyString.substring(TENANT_PREFIX_LENGTH + 3)); + midkeyPosition += rowNum; + foundTenant = true; + LOG.info("Midkey analysis:"); + LOG.info(" - Located in tenant: {}", midkeyTenant); + LOG.info(" - Row number within tenant: {}/{}", rowNum, rowsPerTenant[i]); + LOG.info(" - Position in file: {}/{} ({}%)", midkeyPosition, totalRows, + String.format("%.1f", (midkeyPosition * 100.0) / totalRows)); + LOG.info(" - Target midpoint: {}/{} (50.0%)", totalRows / 2, totalRows); + LOG.info(" - Deviation from midpoint: {}%", + String.format("%.1f", Math.abs(midkeyPosition - totalRows / 2) * 100.0 / totalRows)); + break; + } else { + midkeyPosition += rowsPerTenant[i]; + } + } + + assertTrue("Midkey tenant should be found in tenant list", foundTenant); + + // First and last keys for comparison + if (reader.getFirstRowKey().isPresent() && reader.getLastRowKey().isPresent()) { + String firstKey = Bytes.toString(reader.getFirstRowKey().get()); + String lastKey = Bytes.toString(reader.getLastRowKey().get()); + LOG.info("First key: {}", firstKey); + LOG.info("Last key: {}", lastKey); + LOG.info("Midkey comparison - first: {}, midkey: {}, last: {}", firstKey, midkeyString, + lastKey); + } + + LOG.info("Total rows in dataset: {}", totalRows); + } + } + + /** + * Comprehensive data integrity verification using scanning operations. This method tests various + * scanning scenarios to ensure data integrity after split. + */ + private void verifyDataIntegrityWithScanning(TableName tableName, String[] tenants, + int[] rowsPerTenant) throws Exception { + LOG.info("=== Comprehensive Scanning Verification After Split ==="); + + try (Connection conn = ConnectionFactory.createConnection(testUtil.getConfiguration()); + Table table = conn.getTable(tableName)) { + + // Test 1: Full table scan verification + LOG.info("Test 1: Full table scan verification"); + verifyFullTableScanAfterSplit(table, tenants, rowsPerTenant); + + // Test 2: Tenant-specific scan verification + LOG.info("Test 2: Tenant-specific scan verification"); + verifyTenantSpecificScansAfterSplit(table, tenants, rowsPerTenant); + + // Test 3: Cross-region boundary scanning + LOG.info("Test 3: Cross-region boundary scanning"); + verifyCrossRegionBoundaryScanning(table, tenants, rowsPerTenant); + + // Test 4: Edge cases and tenant isolation + LOG.info("Test 4: Edge cases and tenant isolation verification"); + verifyEdgeCasesAfterSplit(table, tenants, rowsPerTenant); + + LOG.info("Comprehensive scanning verification completed successfully"); + } + } + + /** + * Verify full table scan returns all data correctly after split. + */ + private void verifyFullTableScanAfterSplit(Table table, String[] tenants, int[] rowsPerTenant) + throws IOException { + LOG.info("Performing full table scan to verify all data after split"); + + org.apache.hadoop.hbase.client.Scan fullScan = new org.apache.hadoop.hbase.client.Scan(); + fullScan.addColumn(FAMILY, QUALIFIER); + + try (org.apache.hadoop.hbase.client.ResultScanner scanner = table.getScanner(fullScan)) { + int totalRowsScanned = 0; + int[] tenantRowCounts = new int[tenants.length]; + // Track seen rows per tenant to identify gaps later + @SuppressWarnings("unchecked") + java.util.Set[] seenRowsPerTenant = new java.util.HashSet[tenants.length]; + for (int i = 0; i < tenants.length; i++) { + seenRowsPerTenant[i] = new java.util.HashSet<>(); + } + + String previousRowKey = null; + + for (org.apache.hadoop.hbase.client.Result result : scanner) { + if (result.isEmpty()) { + LOG.warn("Empty result encountered during scan"); + continue; + } + + String rowKey = Bytes.toString(result.getRow()); + + // Verify row ordering + if (previousRowKey != null) { + assertTrue( + "Rows should be in lexicographic order: " + previousRowKey + " should be <= " + rowKey, + Bytes.compareTo(Bytes.toBytes(previousRowKey), Bytes.toBytes(rowKey)) <= 0); + } + + String tenantPrefix = rowKey.substring(0, TENANT_PREFIX_LENGTH); + + // Find which tenant this row belongs to + int tenantIndex = -1; + for (int i = 0; i < tenants.length; i++) { + if (tenants[i].equals(tenantPrefix)) { + tenantIndex = i; + break; + } + } + + if (tenantIndex == -1) { + fail("Found row with unknown tenant prefix: " + rowKey); + } + + // Verify data integrity + byte[] cellValue = result.getValue(FAMILY, QUALIFIER); + if (cellValue == null) { + fail("Missing value for row: " + rowKey); + } + + String actualValue = Bytes.toString(cellValue); + if (!actualValue.contains(tenantPrefix)) { + fail("Tenant data mixing detected in full scan: Row " + rowKey + " expected tenant " + + tenantPrefix + " but got value " + actualValue); + } + + tenantRowCounts[tenantIndex]++; + seenRowsPerTenant[tenantIndex].add(rowKey); + + // Log every 1000th row for progress tracking + if (totalRowsScanned % 1000 == 0) { + LOG.info("Scanned {} rows so far, current row: {}", totalRowsScanned, rowKey); + } + + previousRowKey = rowKey; + totalRowsScanned++; + } + + // Detailed logging of per-tenant counts before assertions + StringBuilder sb = new StringBuilder(); + sb.append("Per-tenant scan results: "); + for (int i = 0; i < tenants.length; i++) { + sb.append(tenants[i]).append("=").append(tenantRowCounts[i]).append("/") + .append(rowsPerTenant[i]).append(", "); + } + sb.append("total=").append(totalRowsScanned); + LOG.info(sb.toString()); + + // Verify total row count + int expectedTotal = Arrays.stream(rowsPerTenant).sum(); + if (totalRowsScanned != expectedTotal) { + LOG.error("Row count mismatch in full scan:"); + LOG.error(" Expected: {}", expectedTotal); + LOG.error(" Scanned: {}", totalRowsScanned); + + // Log missing rows per tenant + for (int i = 0; i < tenants.length; i++) { + if (tenantRowCounts[i] != rowsPerTenant[i]) { + java.util.List missing = new java.util.ArrayList<>(); + for (int r = 0; r < rowsPerTenant[i]; r++) { + String expectedKey = String.format("%srow%05d", tenants[i], r); + if (!seenRowsPerTenant[i].contains(expectedKey)) { + missing.add(expectedKey); + } + } + LOG.error("Missing rows for tenant {} ({} missing): {}", tenants[i], missing.size(), + missing.size() <= 10 ? missing : missing.subList(0, 10) + "..."); + } + } + + fail("Full scan should return all rows after split. Expected: " + expectedTotal + ", Got: " + + totalRowsScanned); + } + + // Verify per-tenant row counts + for (int i = 0; i < tenants.length; i++) { + if (tenantRowCounts[i] != rowsPerTenant[i]) { + LOG.error("Row count mismatch for tenant {} in full scan: expected {}, got {}", + tenants[i], rowsPerTenant[i], tenantRowCounts[i]); + + // Log some missing rows for debugging + java.util.List missing = new java.util.ArrayList<>(); + for (int r = 0; r < rowsPerTenant[i]; r++) { + String expectedKey = String.format("%srow%05d", tenants[i], r); + if (!seenRowsPerTenant[i].contains(expectedKey)) { + missing.add(expectedKey); + if (missing.size() >= 5) { + break; // Show first 5 missing rows + } + } + } + LOG.error("Sample missing rows for tenant {}: {}", tenants[i], missing); + + fail("Row count mismatch for tenant " + tenants[i] + " in full scan: expected " + + rowsPerTenant[i] + ", got " + tenantRowCounts[i]); + } + } + + LOG.info("Full table scan verified successfully: {}/{} rows scanned", totalRowsScanned, + expectedTotal); + } + } + + /** + * Verify tenant-specific scans work correctly after split. + */ + private void verifyTenantSpecificScansAfterSplit(Table table, String[] tenants, + int[] rowsPerTenant) throws IOException { + LOG.info("Verifying tenant-specific scans after split"); + + for (int tenantIndex = 0; tenantIndex < tenants.length; tenantIndex++) { + String tenant = tenants[tenantIndex]; + int expectedRows = rowsPerTenant[tenantIndex]; + + LOG.info("Testing tenant-specific scan for tenant {}: expecting {} rows", tenant, + expectedRows); + + org.apache.hadoop.hbase.client.Scan tenantScan = new org.apache.hadoop.hbase.client.Scan(); + tenantScan.addColumn(FAMILY, QUALIFIER); + tenantScan.withStartRow(Bytes.toBytes(tenant + "row")); + tenantScan.withStopRow(Bytes.toBytes(tenant + "row" + "\uFFFF")); + + try ( + org.apache.hadoop.hbase.client.ResultScanner tenantScanner = table.getScanner(tenantScan)) { + int tenantRowCount = 0; + List foundRows = new ArrayList<>(); + + for (org.apache.hadoop.hbase.client.Result result : tenantScanner) { + String rowKey = Bytes.toString(result.getRow()); + foundRows.add(rowKey); + + if (!rowKey.startsWith(tenant)) { + fail("Tenant scan violation after split: Found row " + rowKey + " in scan for tenant " + + tenant); + } + + // Verify data integrity for this row + byte[] cellValue = result.getValue(FAMILY, QUALIFIER); + if (cellValue == null) { + fail("Missing value for tenant row: " + rowKey); + } + + String actualValue = Bytes.toString(cellValue); + if (!actualValue.contains(tenant)) { + fail("Tenant data corruption after split: Row " + rowKey + " expected tenant " + tenant + + " but got value " + actualValue); + } + + tenantRowCount++; + } + + if (tenantRowCount != expectedRows) { + LOG.error("Row count mismatch for tenant {} after split:", tenant); + LOG.error(" Expected: {}", expectedRows); + LOG.error(" Found: {}", tenantRowCount); + LOG.error(" Found rows: {}", foundRows); + } + + assertEquals("Row count mismatch for tenant " + tenant + " after split", expectedRows, + tenantRowCount); + + LOG.info("Tenant {} scan successful after split: {}/{} rows verified", tenant, + tenantRowCount, expectedRows); + } + } + + LOG.info("All tenant-specific scans verified successfully after split"); + } + + /** + * Verify scanning across region boundaries works correctly. + */ + private void verifyCrossRegionBoundaryScanning(Table table, String[] tenants, int[] rowsPerTenant) + throws IOException { + LOG.info("Verifying cross-region boundary scanning after split"); + + // Test scanning across the split point + // Find a range that likely spans both regions + String firstTenant = tenants[0]; + String lastTenant = tenants[tenants.length - 1]; + + org.apache.hadoop.hbase.client.Scan crossRegionScan = new org.apache.hadoop.hbase.client.Scan(); + crossRegionScan.addColumn(FAMILY, QUALIFIER); + crossRegionScan.withStartRow(Bytes.toBytes(firstTenant + "row000")); + crossRegionScan.withStopRow(Bytes.toBytes(lastTenant + "row999")); + + try (org.apache.hadoop.hbase.client.ResultScanner scanner = table.getScanner(crossRegionScan)) { + int totalRowsScanned = 0; + String previousRowKey = null; + + for (org.apache.hadoop.hbase.client.Result result : scanner) { + String rowKey = Bytes.toString(result.getRow()); + + // Verify row ordering is maintained across regions + if (previousRowKey != null) { + assertTrue("Row ordering should be maintained across regions: " + previousRowKey + + " should be <= " + rowKey, + Bytes.compareTo(Bytes.toBytes(previousRowKey), Bytes.toBytes(rowKey)) <= 0); + } + + // Verify data integrity + byte[] cellValue = result.getValue(FAMILY, QUALIFIER); + if (cellValue == null) { + fail("Missing value in cross-region scan for row: " + rowKey); + } + + String tenantPrefix = rowKey.substring(0, TENANT_PREFIX_LENGTH); + String actualValue = Bytes.toString(cellValue); + if (!actualValue.contains(tenantPrefix)) { + fail("Data corruption in cross-region scan: Row " + rowKey + " expected tenant " + + tenantPrefix + " but got value " + actualValue); + } + + previousRowKey = rowKey; + totalRowsScanned++; + } + + assertTrue("Cross-region scan should find data", totalRowsScanned > 0); + LOG.info("Cross-region boundary scan verified: {} rows scanned with proper ordering", + totalRowsScanned); + } + } + + /** + * Verify edge cases and tenant isolation after split. + */ + private void verifyEdgeCasesAfterSplit(Table table, String[] tenants, int[] rowsPerTenant) + throws IOException { + LOG.info("Verifying edge cases and tenant isolation after split"); + + // Test 1: Non-existent tenant scan + LOG.info("Testing scan with non-existent tenant prefix"); + String nonExistentTenant = "ZZZ"; + org.apache.hadoop.hbase.client.Scan nonExistentScan = new org.apache.hadoop.hbase.client.Scan(); + nonExistentScan.addColumn(FAMILY, QUALIFIER); + nonExistentScan.withStartRow(Bytes.toBytes(nonExistentTenant + "row")); + nonExistentScan.withStopRow(Bytes.toBytes(nonExistentTenant + "row" + "\uFFFF")); + + try (org.apache.hadoop.hbase.client.ResultScanner scanner = table.getScanner(nonExistentScan)) { + int rowCount = 0; + for (org.apache.hadoop.hbase.client.Result result : scanner) { + rowCount++; + } + assertEquals("Non-existent tenant scan should return no results after split", 0, rowCount); + } + + // Test 2: Tenant boundary isolation + LOG.info("Testing tenant boundary isolation after split"); + for (int i = 0; i < tenants.length - 1; i++) { + String tenant1 = tenants[i]; + String tenant2 = tenants[i + 1]; + + // Scan from last row of tenant1 to first row of tenant2 + org.apache.hadoop.hbase.client.Scan boundaryScan = new org.apache.hadoop.hbase.client.Scan(); + boundaryScan.addColumn(FAMILY, QUALIFIER); + boundaryScan + .withStartRow(Bytes.toBytes(tenant1 + "row" + String.format("%05d", rowsPerTenant[i] - 1))); + boundaryScan.withStopRow(Bytes.toBytes(tenant2 + "row001")); + + try (org.apache.hadoop.hbase.client.ResultScanner scanner = table.getScanner(boundaryScan)) { + boolean foundTenant1 = false; + boolean foundTenant2 = false; + + for (org.apache.hadoop.hbase.client.Result result : scanner) { + String rowKey = Bytes.toString(result.getRow()); + + if (rowKey.startsWith(tenant1)) { + foundTenant1 = true; + } else if (rowKey.startsWith(tenant2)) { + foundTenant2 = true; + } else { + fail("Unexpected tenant in boundary scan after split: " + rowKey); + } + } + + // We should find data from both tenants at the boundary + assertTrue("Should find tenant " + tenant1 + " data in boundary scan", foundTenant1); + if (rowsPerTenant[i + 1] > 0) { + assertTrue("Should find tenant " + tenant2 + " data in boundary scan", foundTenant2); + } + } + } + + LOG.info("Edge cases and tenant isolation verification completed successfully"); + } + + /** + * Verify data integrity after split using GET operations. + */ + private void verifyDataIntegrityAfterSplit(TableName tableName, String[] tenants, + int[] rowsPerTenant) throws Exception { + LOG.info("Verifying data integrity with GET operations"); + + try (Connection conn = ConnectionFactory.createConnection(testUtil.getConfiguration()); + Table table = conn.getTable(tableName)) { + + int totalRowsVerified = 0; + + for (int tenantIndex = 0; tenantIndex < tenants.length; tenantIndex++) { + String tenant = tenants[tenantIndex]; + int rowsForThisTenant = rowsPerTenant[tenantIndex]; + + for (int i = 0; i < rowsForThisTenant; i++) { + String rowKey = String.format("%srow%05d", tenant, i); + String expectedValue = String.format("value_tenant-%s_row-%05d", tenant, i); + + Get get = new Get(Bytes.toBytes(rowKey)); + get.addColumn(FAMILY, QUALIFIER); + + Result result = table.get(get); + assertFalse("Result should not be empty for row: " + rowKey, result.isEmpty()); + + byte[] actualValue = result.getValue(FAMILY, QUALIFIER); + String actualValueStr = Bytes.toString(actualValue); + assertEquals("Value mismatch for row " + rowKey, expectedValue, actualValueStr); + totalRowsVerified++; + } + } + + int expectedTotal = Arrays.stream(rowsPerTenant).sum(); + assertEquals("All rows should be verified", expectedTotal, totalRowsVerified); + LOG.info("Data integrity verified: {}/{} rows", totalRowsVerified, expectedTotal); + } + } + + /** + * Find all HFiles created for the test table. + */ + private List findHFilePaths(TableName tableName) throws IOException { + List hfilePaths = new ArrayList<>(); + + Path rootDir = testUtil.getDataTestDirOnTestFS(); + Path tableDir = new Path(rootDir, "data/default/" + tableName.getNameAsString()); + + if (testUtil.getTestFileSystem().exists(tableDir)) { + FileStatus[] regionDirs = testUtil.getTestFileSystem().listStatus(tableDir); + + for (FileStatus regionDir : regionDirs) { + if (regionDir.isDirectory() && !regionDir.getPath().getName().startsWith(".")) { + Path familyDir = new Path(regionDir.getPath(), Bytes.toString(FAMILY)); + + if (testUtil.getTestFileSystem().exists(familyDir)) { + FileStatus[] hfiles = testUtil.getTestFileSystem().listStatus(familyDir); + + for (FileStatus hfile : hfiles) { + if ( + !hfile.getPath().getName().startsWith(".") + && !hfile.getPath().getName().endsWith(".tmp") + ) { + hfilePaths.add(hfile.getPath()); + LOG.debug("Found HFile: {} (size: {} bytes)", hfile.getPath().getName(), + hfile.getLen()); + } + } + } + } + } + } + + LOG.info("Found {} HFiles total", hfilePaths.size()); + return hfilePaths; + } + + /** + * Trigger region split and wait for completion using HBaseTestingUtil methods. + */ + private void triggerRegionSplit(String[] tenants, int[] rowsPerTenant, TableName tableName) + throws Exception { + LOG.info("Starting region split for table: {}", tableName); + + // First ensure cluster is healthy and responsive + LOG.info("Checking cluster health before split"); + try { + // Verify cluster is running + assertTrue("Mini cluster should be running", testUtil.getMiniHBaseCluster() != null); + LOG.info("Mini cluster is up and running"); + + // Add more debug info about cluster state + LOG.info("Master is active: {}", testUtil.getMiniHBaseCluster().getMaster().isActiveMaster()); + LOG.info("Number of region servers: {}", + testUtil.getMiniHBaseCluster().getNumLiveRegionServers()); + LOG.info("Master address: {}", testUtil.getMiniHBaseCluster().getMaster().getServerName()); + + // Ensure no regions are in transition before starting split + testUtil.waitUntilNoRegionsInTransition(60000); + + } catch (Exception e) { + LOG.warn("Cluster health check failed: {}", e.getMessage()); + throw new RuntimeException("Cluster is not healthy before split attempt", e); + } + + // Get initial region count and submit split request + LOG.info("Getting initial region count and submitting split"); + try (Connection connection = ConnectionFactory.createConnection(testUtil.getConfiguration()); + Admin admin = connection.getAdmin()) { + + // Ensure table exists and is available + LOG.info("Verifying table exists: {}", tableName); + boolean tableExists = admin.tableExists(tableName); + if (!tableExists) { + throw new RuntimeException("Table " + tableName + " does not exist before split"); + } + + // Ensure table is enabled + if (!admin.isTableEnabled(tableName)) { + LOG.info("Table {} is disabled, enabling it", tableName); + admin.enableTable(tableName); + testUtil.waitTableEnabled(tableName.getName(), 30000); + } + + LOG.info("Table {} exists and is enabled", tableName); + + List regions = admin.getRegions(tableName); + assertEquals("Should have exactly one region before split", 1, regions.size()); + LOG.info("Pre-split verification passed. Table {} has {} region(s)", tableName, + regions.size()); + + RegionInfo regionToSplit = regions.get(0); + LOG.info("Region to split: {} [{} -> {}]", regionToSplit.getEncodedName(), + Bytes.toStringBinary(regionToSplit.getStartKey()), + Bytes.toStringBinary(regionToSplit.getEndKey())); + + // Trigger the split - let HBase choose the split point based on midkey calculation + LOG.info("Submitting split request for table: {}", tableName); + admin.split(tableName); + LOG.info("Split request submitted successfully for table: {}", tableName); + + // Wait a moment for split request to be processed + Thread.sleep(2000); + } + + // Wait for split to complete using HBaseTestingUtil methods with extended timeouts + LOG.info("Waiting for split processing to complete..."); + + // First wait for no regions in transition + boolean splitCompleted = false; + int maxWaitCycles = 12; // 12 * 10 seconds = 2 minutes max + int waitCycle = 0; + + while (!splitCompleted && waitCycle < maxWaitCycles) { + waitCycle++; + LOG.info("Split wait cycle {}/{}: Waiting for regions to stabilize...", waitCycle, + maxWaitCycles); + + try { + // Wait for no regions in transition (10 second timeout per cycle) + testUtil.waitUntilNoRegionsInTransition(10000); + + // Check if split actually completed + try (Connection conn = ConnectionFactory.createConnection(testUtil.getConfiguration()); + Admin checkAdmin = conn.getAdmin()) { + + List currentRegions = checkAdmin.getRegions(tableName); + if (currentRegions.size() > 1) { + splitCompleted = true; + LOG.info("Split completed successfully! Regions after split: {}", + currentRegions.size()); + } else { + LOG.info("Split not yet complete, still {} region(s). Waiting...", + currentRegions.size()); + Thread.sleep(5000); // Wait 5 seconds before next check + } + } + + } catch (Exception e) { + LOG.warn("Error during split wait cycle {}: {}", waitCycle, e.getMessage()); + if (waitCycle == maxWaitCycles) { + throw new RuntimeException("Split failed after maximum wait time", e); + } + Thread.sleep(5000); // Wait before retrying + } + } + + if (!splitCompleted) { + throw new RuntimeException("Region split did not complete within timeout period"); + } + + // Give additional time for the split to fully stabilize + LOG.info("Split completed, waiting for final stabilization..."); + Thread.sleep(3000); + + // Final verification of split completion + LOG.info("Performing final verification of split completion..."); + try (Connection conn = ConnectionFactory.createConnection(testUtil.getConfiguration()); + Admin finalAdmin = conn.getAdmin()) { + + List regionsAfterSplit = finalAdmin.getRegions(tableName); + if (regionsAfterSplit.size() <= 1) { + fail("Region split did not complete successfully. Expected > 1 region, got: " + + regionsAfterSplit.size()); + } + LOG.info("Final verification passed. Regions after split: {}", regionsAfterSplit.size()); + + // Log region details for debugging + for (int i = 0; i < regionsAfterSplit.size(); i++) { + RegionInfo region = regionsAfterSplit.get(i); + LOG.info("Region {}: {} [{} -> {}]", i + 1, region.getEncodedName(), + Bytes.toStringBinary(region.getStartKey()), Bytes.toStringBinary(region.getEndKey())); + } + + LOG.info("Split operation completed successfully."); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBytesReadFromFs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBytesReadFromFs.java index 86decc94b68f..efdef048830f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBytesReadFromFs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBytesReadFromFs.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.io.hfile; +import java.io.DataInput; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -326,10 +327,6 @@ private void readBloomFilters(Path path, BloomType bt, byte[] key, KeyValue keyV throws IOException { Assert.assertTrue(keyValue == null || key == null); - // Assert that the bloom filter index was read and it's size is accounted in bytes read from - // fs - readLoadOnOpenDataSection(path, true); - CacheConfig cacheConf = new CacheConfig(conf); StoreFileInfo storeFileInfo = StoreFileInfo.createStoreFileInfoForHFile(conf, fs, path, true); HStoreFile sf = new HStoreFile(storeFileInfo, bt, cacheConf); @@ -343,37 +340,94 @@ private void readBloomFilters(Path path, BloomType bt, byte[] key, KeyValue keyV ThreadLocalServerSideScanMetrics.getBlockReadOpsCountAndReset(); StoreFileReader reader = sf.getReader(); - BloomFilter bloomFilter = reader.getGeneralBloomFilter(); - Assert.assertTrue(bloomFilter instanceof CompoundBloomFilter); - CompoundBloomFilter cbf = (CompoundBloomFilter) bloomFilter; + HFile.Reader hfileReader = reader.getHFileReader(); + + if (hfileReader instanceof AbstractMultiTenantReader) { + AbstractMultiTenantReader mtReader = (AbstractMultiTenantReader) hfileReader; + byte[][] sectionIds = mtReader.getAllTenantSectionIds(); + Assert.assertTrue("Expected at least one tenant section", sectionIds.length > 0); + + long totalBloomKeys = 0; + long expectedBytesRead = 0; + int expectedBlockReads = 0; + + for (byte[] sectionId : sectionIds) { + try (AbstractMultiTenantReader.SectionReaderLease lease = + mtReader.getSectionReader(sectionId)) { + if (lease == null) { + continue; + } + HFileReaderImpl sectionReader = lease.getReader(); + DataInput bloomMeta = sectionReader.getGeneralBloomFilterMetadata(); + Assert.assertNotNull("Expected bloom metadata for section", bloomMeta); + BloomFilter bloomFilter = + BloomFilterFactory.createFromMeta(bloomMeta, sectionReader, null); + Assert.assertTrue(bloomFilter instanceof CompoundBloomFilter); + CompoundBloomFilter cbf = (CompoundBloomFilter) bloomFilter; + + totalBloomKeys += cbf.getKeyCount(); + + HFileBlockIndex.BlockIndexReader index = cbf.getBloomIndex(); + Assert.assertTrue("Bloom index should have at least one block", + index.getRootBlockCount() > 0); + + // Read the first bloom block for this section + HFileBlock bloomBlock = cbf.getBloomBlock(0); + long bytesRead = bloomBlock.getOnDiskSizeWithHeader(); + if (bloomBlock.getNextBlockOnDiskSize() > 0) { + bytesRead += HFileBlock.headerSize(true); + } + Assert.assertEquals(BlockType.BLOOM_CHUNK, bloomBlock.getBlockType()); + bloomBlock.release(); + + expectedBytesRead += bytesRead; + expectedBlockReads++; + } + } - // Get the bloom filter index reader - HFileBlockIndex.BlockIndexReader index = cbf.getBloomIndex(); - int block; + reader.close(true); - // Search for the key in the bloom filter index - if (keyValue != null) { - block = index.rootBlockContainingKey(keyValue); + Assert.assertEquals("Bloom key count mismatch", keyList.size(), totalBloomKeys); + Assert.assertEquals(expectedBytesRead, + ThreadLocalServerSideScanMetrics.getBytesReadFromFsAndReset()); + Assert.assertEquals(expectedBlockReads, + ThreadLocalServerSideScanMetrics.getBlockReadOpsCountAndReset()); } else { - byte[] row = key; - block = index.rootBlockContainingKey(row, 0, row.length); - } + // Assert that the bloom filter index was read and accounted in metrics + readLoadOnOpenDataSection(path, true); - // Read the bloom block from FS - HFileBlock bloomBlock = cbf.getBloomBlock(block); - long bytesRead = bloomBlock.getOnDiskSizeWithHeader(); - if (bloomBlock.getNextBlockOnDiskSize() > 0) { - bytesRead += HFileBlock.headerSize(true); - } - // Asser that the block read is a bloom block - Assert.assertEquals(bloomBlock.getBlockType(), BlockType.BLOOM_CHUNK); - bloomBlock.release(); + BloomFilter bloomFilter = reader.getGeneralBloomFilter(); + Assert.assertTrue(bloomFilter instanceof CompoundBloomFilter); + CompoundBloomFilter cbf = (CompoundBloomFilter) bloomFilter; - // Close the reader - reader.close(true); + // Get the bloom filter index reader + HFileBlockIndex.BlockIndexReader index = cbf.getBloomIndex(); + int block; - Assert.assertEquals(bytesRead, ThreadLocalServerSideScanMetrics.getBytesReadFromFsAndReset()); - Assert.assertEquals(1, ThreadLocalServerSideScanMetrics.getBlockReadOpsCountAndReset()); + // Search for the key in the bloom filter index + if (keyValue != null) { + block = index.rootBlockContainingKey(keyValue); + } else { + byte[] row = key; + block = index.rootBlockContainingKey(row, 0, row.length); + } + + // Read the bloom block from FS + HFileBlock bloomBlock = cbf.getBloomBlock(block); + long bytesRead = bloomBlock.getOnDiskSizeWithHeader(); + if (bloomBlock.getNextBlockOnDiskSize() > 0) { + bytesRead += HFileBlock.headerSize(true); + } + // Asser that the block read is a bloom block + Assert.assertEquals(bloomBlock.getBlockType(), BlockType.BLOOM_CHUNK); + bloomBlock.release(); + + // Close the reader + reader.close(true); + + Assert.assertEquals(bytesRead, ThreadLocalServerSideScanMetrics.getBytesReadFromFsAndReset()); + Assert.assertEquals(1, ThreadLocalServerSideScanMetrics.getBlockReadOpsCountAndReset()); + } } private void writeBloomFilters(Path path, BloomType bt, int bloomBlockByteSize) @@ -385,9 +439,6 @@ private void writeBloomFilters(Path path, BloomType bt, int bloomBlockByteSize) .withCompression(Compression.Algorithm.NONE).build(); StoreFileWriter w = new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) .withBloomType(bt).withFilePath(path).build(); - Assert.assertTrue(w.hasGeneralBloom()); - Assert.assertTrue(w.getGeneralBloomWriter() instanceof CompoundBloomFilterWriter); - CompoundBloomFilterWriter cbbf = (CompoundBloomFilterWriter) w.getGeneralBloomWriter(); byte[] cf = Bytes.toBytes("cf"); byte[] cq = Bytes.toBytes("cq"); for (int i = 0; i < NUM_KEYS; i++) { @@ -400,7 +451,6 @@ private void writeBloomFilters(Path path, BloomType bt, int bloomBlockByteSize) keyList.add(keyBytes); keyValues.add(keyValue); } - Assert.assertEquals(keyList.size(), cbbf.getKeyCount()); w.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index fbc7f3bedd1d..cbbd0d43d5ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -156,6 +156,17 @@ private void fillByteBuffAllocator(ByteBuffAllocator alloc, int bufCount) { Assert.assertEquals(alloc.getFreeBufferCount(), bufCount); } + private BlockCacheKey cacheKeyFor(HFile.Reader reader, HFileBlock block) { + String hfileName = block.getHFileContext().getHFileName(); + if (hfileName == null) { + hfileName = reader.getName(); + if (reader instanceof AbstractMultiTenantReader && !hfileName.endsWith("#")) { + hfileName = hfileName + "#"; + } + } + return new BlockCacheKey(hfileName, block.getOffset()); + } + @Test public void testReaderWithoutBlockCache() throws Exception { int bufCount = 32; @@ -184,8 +195,8 @@ public void testReaderWithLRUBlockCache() throws Exception { HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConfig, true, conf); long offset = 0; while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) { - BlockCacheKey key = new BlockCacheKey(storeFilePath.getName(), offset); HFileBlock block = reader.readBlock(offset, -1, true, true, false, true, null, null); + BlockCacheKey key = cacheKeyFor(reader, block); offset += block.getOnDiskSizeWithHeader(); // Ensure the block is an heap one. Cacheable cachedBlock = lru.getBlock(key, false, false, true); @@ -220,7 +231,7 @@ private void assertBytesReadFromCache(boolean isScanMetricsEnabled, DataBlockEnc // Read the first block in HFile from the block cache. final int offset = 0; - BlockCacheKey cacheKey = new BlockCacheKey(storeFilePath.getName(), offset); + BlockCacheKey cacheKey = new BlockCacheKey(reader.getName(), offset); HFileBlock block = (HFileBlock) lru.getBlock(cacheKey, false, false, true); Assert.assertNull(block); @@ -235,14 +246,27 @@ private void assertBytesReadFromCache(boolean isScanMetricsEnabled, DataBlockEnc // Read the first block from the HFile. block = reader.readBlock(offset, -1, true, true, false, true, BlockType.DATA, null); Assert.assertNotNull(block); + cacheKey = cacheKeyFor(reader, block); int bytesReadFromFs = block.getOnDiskSizeWithHeader(); if (block.getNextBlockOnDiskSize() > 0) { bytesReadFromFs += block.headerSize(); } block.release(); // Assert that disk I/O happened to read the first block. + long bytesReadFromFsMetric = ThreadLocalServerSideScanMetrics.getBytesReadFromFsAndReset(); + long effectiveBytesReadFromFs = bytesReadFromFsMetric; + if (isScanMetricsEnabled) { + if (bytesReadFromFsMetric < bytesReadFromFs) { + Assert.assertEquals(bytesReadFromFs, bytesReadFromFsMetric); + } else if (bytesReadFromFsMetric > bytesReadFromFs) { + long initializationBytes = bytesReadFromFsMetric - bytesReadFromFs; + Assert.assertEquals(HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT, + reader.getTrailer().getMajorVersion()); + effectiveBytesReadFromFs -= initializationBytes; + } + } Assert.assertEquals(isScanMetricsEnabled ? bytesReadFromFs : 0, - ThreadLocalServerSideScanMetrics.getBytesReadFromFsAndReset()); + isScanMetricsEnabled ? effectiveBytesReadFromFs : bytesReadFromFsMetric); Assert.assertEquals(0, ThreadLocalServerSideScanMetrics.getBytesReadFromBlockCacheAndReset()); // Read the first block again and assert that it has been cached in the block cache. @@ -313,8 +337,8 @@ public void testReaderWithCombinedBlockCache() throws Exception { HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConfig, true, conf); long offset = 0; while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) { - BlockCacheKey key = new BlockCacheKey(storeFilePath.getName(), offset); HFileBlock block = reader.readBlock(offset, -1, true, true, false, true, null, null); + BlockCacheKey key = cacheKeyFor(reader, block); offset += block.getOnDiskSizeWithHeader(); // Read the cached block. Cacheable cachedBlock = combined.getBlock(key, false, false, true); @@ -1050,8 +1074,8 @@ public void testDBEShipped() throws IOException { Path f = new Path(ROOT_DIR, testName.getMethodName() + "_" + encoding); HFileContext context = new HFileContextBuilder().withIncludesTags(false).withDataBlockEncoding(encoding).build(); - HFileWriterImpl writer = (HFileWriterImpl) HFile.getWriterFactory(conf, cacheConf) - .withPath(fs, f).withFileContext(context).create(); + Writer writer = + HFile.getWriterFactory(conf, cacheConf).withPath(fs, f).withFileContext(context).create(); KeyValue kv = new KeyValue(Bytes.toBytes("testkey1"), Bytes.toBytes("family"), Bytes.toBytes("qual"), Bytes.toBytes("testvalue")); @@ -1114,8 +1138,8 @@ private void testReaderCombinedCache(final String l1CachePolicy) throws Exceptio long offset = 0; Cacheable cachedBlock = null; while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) { - BlockCacheKey key = new BlockCacheKey(storeFilePath.getName(), offset); HFileBlock block = reader.readBlock(offset, -1, true, true, false, true, null, null); + BlockCacheKey key = cacheKeyFor(reader, block); offset += block.getOnDiskSizeWithHeader(); // Read the cached block. cachedBlock = combined.getBlock(key, false, false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java index 3c5be3ce8290..0f12f11e6e54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java @@ -87,14 +87,21 @@ public void testHFilePrettyPrinterNonRootDir() throws Exception { System.setOut(ps); new HFilePrettyPrinter(conf).run(new String[] { "-v", String.valueOf(fileNotInRootDir) }); String result = new String(stream.toByteArray()); - String expectedResult = "Scanning -> " + fileNotInRootDir + "\n" + "Scanned kv count -> 1000\n"; - assertEquals(expectedResult, result); + String expectedFirstLine = "Scanning -> " + fileNotInRootDir + "\n"; + String expectedCountLine = "Scanned kv count -> 1000\n"; + assertTrue("expected to contain start line: '" + expectedFirstLine + "'", + result.contains(expectedFirstLine)); + assertTrue("expected to contain count line: '" + expectedCountLine + "'", + result.contains(expectedCountLine)); + assertTrue("expected start line to appear before count line", + result.indexOf(expectedFirstLine) >= 0 + && result.indexOf(expectedCountLine) > result.indexOf(expectedFirstLine)); } @Test public void testHFilePrettyPrinterRootDir() throws Exception { Path rootPath = CommonFSUtils.getRootDir(conf); - String rootString = rootPath + rootPath.SEPARATOR; + String rootString = rootPath + Path.SEPARATOR; Path fileInRootDir = new Path(rootString + "hfile"); TestHRegionServerBulkLoad.createHFile(fs, fileInRootDir, cf, fam, value, 1000); assertTrue("directory used is a root dir", fileInRootDir.toString().startsWith(rootString)); @@ -105,8 +112,15 @@ public void testHFilePrettyPrinterRootDir() throws Exception { printer.processFile(fileInRootDir, true); printer.run(new String[] { "-v", String.valueOf(fileInRootDir) }); String result = new String(stream.toByteArray()); - String expectedResult = "Scanning -> " + fileInRootDir + "\n" + "Scanned kv count -> 1000\n"; - assertEquals(expectedResult, result); + String expectedFirstLine = "Scanning -> " + fileInRootDir + "\n"; + String expectedCountLine = "Scanned kv count -> 1000\n"; + assertTrue("expected to contain start line: '" + expectedFirstLine + "'", + result.contains(expectedFirstLine)); + assertTrue("expected to contain count line: '" + expectedCountLine + "'", + result.contains(expectedCountLine)); + assertTrue("expected start line to appear before count line", + result.indexOf(expectedFirstLine) >= 0 + && result.indexOf(expectedCountLine) > result.indexOf(expectedFirstLine)); } @Test @@ -124,8 +138,15 @@ public void testHFilePrettyPrinterSeekFirstRow() throws Exception { new HFilePrettyPrinter(conf) .run(new String[] { "-v", "-w" + firstRowKey, String.valueOf(fileNotInRootDir) }); String result = new String(stream.toByteArray()); - String expectedResult = "Scanning -> " + fileNotInRootDir + "\n" + "Scanned kv count -> 1\n"; - assertEquals(expectedResult, result); + String expectedFirstLine = "Scanning -> " + fileNotInRootDir + "\n"; + String expectedCountLine = "Scanned kv count -> 1\n"; + assertTrue("expected to contain start line: '" + expectedFirstLine + "'", + result.contains(expectedFirstLine)); + assertTrue("expected to contain count line: '" + expectedCountLine + "'", + result.contains(expectedCountLine)); + assertTrue("expected start line to appear before count line", + result.indexOf(expectedFirstLine) >= 0 + && result.indexOf(expectedCountLine) > result.indexOf(expectedFirstLine)); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderImpl.java index 6c84312cf599..096d3794be09 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderImpl.java @@ -98,8 +98,7 @@ public void testRecordBlockSize() throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); HFile.Reader reader = HFile.createReader(fs, p, CacheConfig.DISABLED, true, conf); - try (HFileReaderImpl.HFileScannerImpl scanner = - (HFileReaderImpl.HFileScannerImpl) reader.getScanner(conf, true, true, false)) { + try (HFileScanner scanner = reader.getScanner(conf, true, true, false)) { scanner.seekTo(); scanner.recordBlockSize( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileV4PrettyPrinter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileV4PrettyPrinter.java new file mode 100644 index 000000000000..d06f13ea26b3 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileV4PrettyPrinter.java @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tests for HFilePrettyPrinter with HFile v4 multi-tenant features. This test validates that the + * pretty printer correctly handles v4 HFiles with multi-tenant capabilities including tenant + * information display, tenant-aware block analysis, and comprehensive output formatting. + */ +@Category({ IOTests.class, MediumTests.class }) +public class TestHFileV4PrettyPrinter { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHFileV4PrettyPrinter.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestHFileV4PrettyPrinter.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + private static final int TENANT_PREFIX_LENGTH = 3; + private static final String[] TENANTS = { "T01", "T02", "T03" }; + private static final int TEST_TIMEOUT_MS = 120000; // 2 minutes + + private static FileSystem fs; + private static Configuration conf; + private static final byte[] cf = Bytes.toBytes("cf"); + private static final byte[] fam = Bytes.toBytes("fam"); + private static PrintStream original; + private static PrintStream ps; + private static ByteArrayOutputStream stream; + + @Before + public void setup() throws Exception { + conf = UTIL.getConfiguration(); + + // Configure HFile v4 multi-tenant settings + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + conf.setInt(MultiTenantHFileWriter.TENANT_PREFIX_LENGTH, TENANT_PREFIX_LENGTH); + + // Runs on local filesystem. Test does not need sync. Turn off checks. + conf.setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, false); + + // Start mini cluster for v4 HFile creation + UTIL.startMiniCluster(1); + + fs = UTIL.getTestFileSystem(); + stream = new ByteArrayOutputStream(); + ps = new PrintStream(stream); + original = System.out; + + LOG.info("Setup complete with HFile v4 configuration"); + } + + @After + public void teardown() throws Exception { + System.setOut(original); + if (UTIL != null) { + UTIL.shutdownMiniCluster(); + } + } + + /** + * Create a v4 multi-tenant HFile with test data. + */ + private Path createV4HFile(String testName, int rowCount) throws Exception { + TableName tableName = TableName.valueOf(testName + "_" + System.currentTimeMillis()); + + try (Admin admin = UTIL.getAdmin()) { + // Create table with multi-tenant configuration + TableDescriptorBuilder tableBuilder = TableDescriptorBuilder.newBuilder(tableName); + + // Set multi-tenant properties + tableBuilder.setValue(MultiTenantHFileWriter.TABLE_TENANT_PREFIX_LENGTH, + String.valueOf(TENANT_PREFIX_LENGTH)); + tableBuilder.setValue(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED, "true"); + + // Configure column family for HFile v4 + ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(cf); + cfBuilder.setValue(HFile.FORMAT_VERSION_KEY, + String.valueOf(HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT)); + tableBuilder.setColumnFamily(cfBuilder.build()); + + admin.createTable(tableBuilder.build()); + UTIL.waitTableAvailable(tableName); + + // Write test data with tenant prefixes + try (Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(tableName)) { + + List puts = new ArrayList<>(); + int rowsPerTenant = rowCount / TENANTS.length; + + for (String tenantId : TENANTS) { + for (int i = 0; i < rowsPerTenant; i++) { + String rowKey = String.format("%srow%03d", tenantId, i); + Put put = new Put(Bytes.toBytes(rowKey)); + String cellValue = String.format("value_tenant-%s_row-%03d", tenantId, i); + put.addColumn(cf, fam, Bytes.toBytes(cellValue)); + puts.add(put); + } + } + + table.put(puts); + LOG.info("Wrote {} rows to v4 multi-tenant table {}", puts.size(), tableName); + } + + // Flush to create HFile v4 + UTIL.flush(tableName); + Thread.sleep(1000); // Wait for flush to complete + + // Find the created HFile + List hfiles = + UTIL.getHBaseCluster().getRegions(tableName).get(0).getStore(cf).getStorefiles().stream() + .map(sf -> sf.getPath()).collect(java.util.stream.Collectors.toList()); + + assertTrue("Should have created at least one HFile", !hfiles.isEmpty()); + Path originalHfilePath = hfiles.get(0); + + LOG.info("Found original v4 HFile: {}", originalHfilePath); + + // Copy HFile to test data directory before table cleanup + Path testDataDir = UTIL.getDataTestDir(testName); + Path copiedHfilePath = new Path(testDataDir, "hfile_v4_" + System.currentTimeMillis()); + + // Use FileUtil to copy the file + org.apache.hadoop.fs.FileUtil.copy(fs, originalHfilePath, fs, copiedHfilePath, false, conf); + + LOG.info("Copied v4 HFile from {} to {}", originalHfilePath, copiedHfilePath); + + // Verify the copied file is actually v4 + try (HFile.Reader reader = + HFile.createReader(fs, copiedHfilePath, CacheConfig.DISABLED, true, conf)) { + int version = reader.getTrailer().getMajorVersion(); + assertEquals("Should be HFile v4", HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT, version); + LOG.info("Verified copied HFile v4 format: version {}", version); + } + + // Clean up table (original HFiles will be deleted but our copy is safe) + admin.disableTable(tableName); + admin.deleteTable(tableName); + + return copiedHfilePath; + } + } + + /** + * Comprehensive test for HFilePrettyPrinter with HFile v4 multi-tenant features. This test + * validates: - HFile v4 format detection and verification - All command-line options + * functionality (-m, -p, -v, -t, -b, -h, -s, -d) - Multi-tenant specific output including tenant + * information - Tenant boundary detection and display - Block-level analysis with v4 multi-tenant + * structure - Key/value pair display with tenant context + */ + @Test(timeout = TEST_TIMEOUT_MS) + public void testComprehensiveV4Output() throws Exception { + Path testFile = createV4HFile("hfile_comprehensive_v4", 90); + + // First, verify the created file is actually v4 format (version detection) + try (HFile.Reader reader = HFile.createReader(fs, testFile, CacheConfig.DISABLED, true, conf)) { + int majorVersion = reader.getTrailer().getMajorVersion(); + LOG.info("Detected HFile version: {} (v4 threshold: {})", majorVersion, + HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + assertTrue("Test file should be v4", + majorVersion == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + } + + System.setOut(ps); + HFilePrettyPrinter printer = new HFilePrettyPrinter(conf); + + LOG.info("=== COMPREHENSIVE HFILE V4 OUTPUT TEST ==="); + LOG.info("Testing file: {}", testFile); + + // Run with ALL possible options for comprehensive output + printer.run(new String[] { "-m", // metadata + "-p", // print key/value pairs + "-v", // verbose + "-t", // tenant info (v4 specific) + "-b", // block index + "-h", // block headers + "-s", // statistics/histograms + "-d", // detailed output + "-f", testFile.toString() }); + + String comprehensiveResult = stream.toString(); + + LOG.info("=== FULL HFILE V4 COMPREHENSIVE OUTPUT START ==="); + LOG.info("\n{}", comprehensiveResult); + LOG.info("=== FULL HFILE V4 COMPREHENSIVE OUTPUT END ==="); + + // Verify all expected sections are present + assertTrue("Should contain trailer information", comprehensiveResult.contains("Trailer:")); + assertTrue("Should contain file info", comprehensiveResult.contains("Fileinfo:")); + assertTrue("Should contain v4-specific information", + comprehensiveResult.contains("HFile v4 Specific Information:")); + assertTrue("Should contain tenant information", + comprehensiveResult.contains("Tenant Information:")); + assertTrue("Should contain block index", comprehensiveResult.contains("Block Index:")); + assertTrue("Should contain block headers", comprehensiveResult.contains("Block Headers:")); + assertTrue("Should contain key/value pairs", comprehensiveResult.contains("K: ")); + assertTrue("Should contain tenant boundaries", + comprehensiveResult.contains("--- Start of tenant section:") + || comprehensiveResult.contains("Scanning multi-tenant HFile v4")); + + // Verify tenant-specific data is present + for (String tenant : TENANTS) { + assertTrue("Should contain data for tenant " + tenant, + comprehensiveResult.contains(tenant + "row")); + } + + LOG.info("Comprehensive V4 test completed successfully"); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java index b01717dfa1f8..d93003a43cfe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.AbstractMultiTenantReader; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -59,6 +60,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheTestUtils; +import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; @@ -591,7 +593,7 @@ public void testCacheOnReadColdFile() throws Exception { this.blockCache = initializeTestEnvironment(); // hStoreFiles[3] is a cold file. the blocks should not get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(3); - BlockCacheKey cacheKey = new BlockCacheKey(hStoreFile.getPath(), 0, true, BlockType.DATA); + BlockCacheKey cacheKey = createBlockCacheKey(hStoreFile, 0, BlockType.DATA); testCacheOnRead(hStoreFile, cacheKey, -1, false); } @@ -600,8 +602,7 @@ public void testCacheOnReadHotFile() throws Exception { this.blockCache = initializeTestEnvironment(); // hStoreFiles[0] is a hot file. the blocks should get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(0); - BlockCacheKey cacheKey = - new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA); + BlockCacheKey cacheKey = createBlockCacheKey(hStoreFile, 0, BlockType.DATA); testCacheOnRead(hStoreFile, cacheKey, -1, true); } @@ -693,6 +694,40 @@ private void testDataTieringMethodWithKeyNoException(DataTieringMethodCallerWith testDataTieringMethodWithKey(caller, key, expectedResult, null); } + private BlockCacheKey createBlockCacheKey(HStoreFile hStoreFile, long blockOffset, + BlockType blockType) { + StoreFileReader storeFileReader = hStoreFile.getReader(); + HFile.Reader hFileReader = storeFileReader.getHFileReader(); + if ( + storeFileReader.getHFileVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT + && hFileReader instanceof AbstractMultiTenantReader + ) { + AbstractMultiTenantReader multiTenantReader = (AbstractMultiTenantReader) hFileReader; + byte[][] tenantSectionIds = multiTenantReader.getAllTenantSectionIds(); + if (tenantSectionIds != null) { + for (byte[] sectionId : tenantSectionIds) { + Map sectionInfo = multiTenantReader.getSectionInfo(sectionId); + if (sectionInfo == null || !Boolean.TRUE.equals(sectionInfo.get("exists"))) { + continue; + } + Object offsetObj = sectionInfo.get("offset"); + Object sizeObj = sectionInfo.get("size"); + if (!(offsetObj instanceof Number) || !(sizeObj instanceof Number)) { + continue; + } + long sectionStart = ((Number) offsetObj).longValue(); + long sectionEnd = sectionStart + ((Number) sizeObj).longValue(); + if (blockOffset >= sectionStart && blockOffset < sectionEnd) { + String sectionSuffix = Bytes.toStringBinary(sectionId); + Path sectionPath = new Path(hStoreFile.getPath().toString() + "#" + sectionSuffix); + return new BlockCacheKey(sectionPath, blockOffset, true, blockType); + } + } + } + } + return new BlockCacheKey(hStoreFile.getPath(), blockOffset, true, blockType); + } + private static BlockCache initializeTestEnvironment() throws IOException { BlockCache blockCache = setupFileSystemAndCache(); setupOnlineRegions(blockCache); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java index bf82a531f199..5c571d4b0f2d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.AbstractMultiTenantReader; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -59,6 +60,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheTestUtils; +import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; @@ -604,7 +606,7 @@ public void testCacheOnReadColdFile() throws Exception { initializeTestEnvironment(); // hStoreFiles[3] is a cold file. the blocks should not get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(3); - BlockCacheKey cacheKey = new BlockCacheKey(hStoreFile.getPath(), 0, true, BlockType.DATA); + BlockCacheKey cacheKey = createBlockCacheKey(hStoreFile, 0, BlockType.DATA); testCacheOnRead(hStoreFile, cacheKey, -1, false); } @@ -613,8 +615,7 @@ public void testCacheOnReadHotFile() throws Exception { initializeTestEnvironment(); // hStoreFiles[0] is a hot file. the blocks should get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(0); - BlockCacheKey cacheKey = - new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA); + BlockCacheKey cacheKey = createBlockCacheKey(hStoreFile, 0, BlockType.DATA); testCacheOnRead(hStoreFile, cacheKey, -1, true); } @@ -705,6 +706,40 @@ private void testDataTieringMethodWithKeyNoException(DataTieringMethodCallerWith testDataTieringMethodWithKey(caller, key, expectedResult, null); } + private BlockCacheKey createBlockCacheKey(HStoreFile hStoreFile, long blockOffset, + BlockType blockType) { + StoreFileReader storeFileReader = hStoreFile.getReader(); + HFile.Reader hFileReader = storeFileReader.getHFileReader(); + if ( + storeFileReader.getHFileVersion() == HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT + && hFileReader instanceof AbstractMultiTenantReader + ) { + AbstractMultiTenantReader multiTenantReader = (AbstractMultiTenantReader) hFileReader; + byte[][] tenantSectionIds = multiTenantReader.getAllTenantSectionIds(); + if (tenantSectionIds != null) { + for (byte[] sectionId : tenantSectionIds) { + Map sectionInfo = multiTenantReader.getSectionInfo(sectionId); + if (sectionInfo == null || !Boolean.TRUE.equals(sectionInfo.get("exists"))) { + continue; + } + Object offsetObj = sectionInfo.get("offset"); + Object sizeObj = sectionInfo.get("size"); + if (!(offsetObj instanceof Number) || !(sizeObj instanceof Number)) { + continue; + } + long sectionStart = ((Number) offsetObj).longValue(); + long sectionEnd = sectionStart + ((Number) sizeObj).longValue(); + if (blockOffset >= sectionStart && blockOffset < sectionEnd) { + String sectionSuffix = Bytes.toStringBinary(sectionId); + Path sectionPath = new Path(hStoreFile.getPath().toString() + "#" + sectionSuffix); + return new BlockCacheKey(sectionPath, blockOffset, true, blockType); + } + } + } + } + return new BlockCacheKey(hStoreFile.getPath(), blockOffset, true, blockType); + } + private static void initializeTestEnvironment() throws IOException { setupFileSystemAndCache(); setupOnlineRegions(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiTenantBloomFilterDelegation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiTenantBloomFilterDelegation.java new file mode 100644 index 000000000000..5219c80d66c5 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiTenantBloomFilterDelegation.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.nio.file.Files; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.hfile.BlockType; +import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.io.hfile.MultiTenantBloomSupport; +import org.apache.hadoop.hbase.io.hfile.MultiTenantHFileWriter; +import org.apache.hadoop.hbase.io.hfile.ReaderContext; +import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestMultiTenantBloomFilterDelegation { + + @Test + public void testRowBloomDelegation() throws Exception { + Configuration conf = HBaseConfiguration.create(); + conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_MULTI_TENANT); + conf.setBoolean(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED, true); + conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true); + + FileSystem fs = FileSystem.getLocal(conf); + Path baseDir = new Path(Files.createTempDirectory("multi-tenant-bloom").toUri()); + Path file = StoreFileWriter.getUniqueFile(fs, baseDir); + + CacheConfig cacheConfig = new CacheConfig(conf); + + Map tableProps = new HashMap<>(); + tableProps.put(MultiTenantHFileWriter.TABLE_MULTI_TENANT_ENABLED, "true"); + tableProps.put(MultiTenantHFileWriter.TABLE_TENANT_PREFIX_LENGTH, "2"); + tableProps.put("BLOOMFILTER", "ROW"); + + HFileContext context = new HFileContextBuilder().withBlockSize(4096) + .withColumnFamily(Bytes.toBytes("cf")).withTableName(Bytes.toBytes("tbl")).build(); + + MultiTenantHFileWriter writer = MultiTenantHFileWriter.create(fs, file, conf, cacheConfig, + tableProps, context, BloomType.ROW, BloomType.ROW, null, true); + + long ts = EnvironmentEdgeManager.currentTime(); + KeyValue tenantOneRow = new KeyValue(Bytes.toBytes("aa-0001"), Bytes.toBytes("cf"), + Bytes.toBytes("q"), ts, Bytes.toBytes("value")); + KeyValue tenantTwoRow = new KeyValue(Bytes.toBytes("bb-0001"), Bytes.toBytes("cf"), + Bytes.toBytes("q"), ts, Bytes.toBytes("value")); + + writer.append(tenantOneRow); + writer.append(tenantTwoRow); + writer.close(); + + ReaderContext contextReader = + new ReaderContextBuilder().withFileSystemAndPath(fs, file).build(); + StoreFileInfo storeFileInfo = StoreFileInfo.createStoreFileInfoForHFile(conf, fs, file, true); + storeFileInfo.initHFileInfo(contextReader); + StoreFileReader reader = storeFileInfo.createReader(contextReader, cacheConfig); + storeFileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader()); + reader.loadFileInfo(); + reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META, new BloomFilterMetrics()); + reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META, new BloomFilterMetrics()); + + byte[] presentRow = Bytes.toBytes("bb-0001"); + byte[] absentRow = Bytes.toBytes("bb-zzzz"); + + HFile.Reader hfileReader = reader.getHFileReader(); + assertTrue(hfileReader instanceof MultiTenantBloomSupport); + MultiTenantBloomSupport bloomSupport = (MultiTenantBloomSupport) hfileReader; + + boolean expectedPresent = + bloomSupport.passesGeneralRowBloomFilter(presentRow, 0, presentRow.length); + assertTrue(expectedPresent); + Scan present = new Scan(new Get(presentRow)); + assertEquals(expectedPresent, reader.passesBloomFilter(present, null)); + + boolean expectedAbsent = + bloomSupport.passesGeneralRowBloomFilter(absentRow, 0, absentRow.length); + Scan absent = new Scan(new Get(absentRow)); + assertEquals(expectedAbsent, reader.passesBloomFilter(absent, null)); + + fs.delete(baseDir, true); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java index 718387ce9f69..858339038418 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java @@ -103,7 +103,7 @@ public static void enableSecurity(Configuration conf) throws IOException { conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName()); conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName()); // Need HFile V3 for tags for security features - conf.setInt(HFile.FORMAT_VERSION_KEY, 3); + conf.setInt(HFile.FORMAT_VERSION_KEY, 4); conf.set(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, "true"); configureSuperuser(conf); }