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:
+ *
+ * Table level settings have highest precedence
+ * Cluster level settings are used as fallback
+ * Default values are used if neither is specified
+ *
+ */
+ // 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:
+ *
+ * Table-level properties have highest precedence
+ * Cluster-level configuration used as fallback
+ * Default values used if neither specified
+ *
+ * @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:
+ *
+ * Setup: Creates table with multi-tenant configuration
+ * Data Writing: Writes data for multiple tenants with distinct prefixes
+ * Flushing: Forces memstore flush to create multi-tenant HFile v4 files
+ * Verification: Tests various read patterns and tenant isolation
+ * Format Validation: Verifies HFile v4 structure and tenant sections
+ *
+ *
+ * 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:
+ *
+ * Create table with multi-tenant configuration
+ * Write test data for {} tenants with varying row counts
+ * Flush memstore to create multi-tenant HFiles
+ * Verify data integrity using GET operations
+ * Verify data using full table SCAN
+ * Verify tenant isolation using tenant-specific scans
+ * Test edge cases and cross-tenant isolation
+ * Validate HFile format and tenant section structure
+ *
+ */
+ @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:
+ *
+ * Setup: Creates table with multi-tenant configuration
+ * Data Writing: Writes large datasets with different tenant distributions
+ * Flushing: Forces memstore flush to create multi-tenant HFile v4 files
+ * Splitting: Tests midkey calculation and file splitting
+ * Verification: Validates split balance and data integrity
+ *
+ */
+@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);
}