From 4ad03c834b1fa9c4917c7f2e11d2858130f51c0e Mon Sep 17 00:00:00 2001 From: Abhishek Kothalikar <99398985+kabhishek4@users.noreply.github.com> Date: Tue, 11 Mar 2025 20:03:25 +0530 Subject: [PATCH 01/35] HBASE-29082: Support for custom meta table name suffix (#6632) (cherry picked from commit 7ab9d52801fc5be72b742582d1732a8f5e602d86) --- .../org/apache/hadoop/hbase/HConstants.java | 12 ++ .../org/apache/hadoop/hbase/TableName.java | 37 +++++- .../hadoop/hbase/TestMetaTableForReplica.java | 114 ++++++++++++++++++ 3 files changed, 160 insertions(+), 3 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableForReplica.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 9af711e7edfd..4f24a739786b 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1665,6 +1665,18 @@ public enum OperationStatusCode { */ public final static boolean REJECT_DECOMMISSIONED_HOSTS_DEFAULT = false; + /** + * Adds a suffix to the meta table name: value=’test’ -> ‘hbase:meta_test’ Added in HBASE-XXXXX to + * support having multiple hbase:meta tables (with distinct names )to enable storage sharing by + * more than one clusters. + */ + public final static String HBASE_META_TABLE_SUFFIX = "hbase.meta.table.suffix"; + + /** + * Default value of {@link #HBASE_META_TABLE_SUFFIX} + */ + public final static String HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE = ""; + private HConstants() { // Can't be instantiated with this ctor. } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index b6d854c13784..73008c7ad5fd 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -17,16 +17,21 @@ */ package org.apache.hadoop.hbase; +import com.google.errorprone.annotations.RestrictedApi; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; import org.apache.commons.lang3.ArrayUtils; +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; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; /** * Immutable POJO class for representing a table name. Which is of the form: <table @@ -44,6 +49,7 @@ */ @InterfaceAudience.Public public final class TableName implements Comparable { + private static final Logger LOG = LoggerFactory.getLogger(TableName.class); /** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */ private static final Set tableCache = new CopyOnWriteArraySet<>(); @@ -65,9 +71,34 @@ public final class TableName implements Comparable { public static final String VALID_USER_TABLE_REGEX = "(?:(?:(?:" + VALID_NAMESPACE_REGEX + "\\" + NAMESPACE_DELIM + ")?)" + "(?:" + VALID_TABLE_QUALIFIER_REGEX + "))"; - /** The hbase:meta table's name. */ - public static final TableName META_TABLE_NAME = - valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + /** + * The name of hbase meta table could either be hbase:meta_xxx or 'hbase:meta' otherwise. Config + * hbase.meta.table.suffix will govern the decision of adding suffix to the habase:meta + */ + public static final TableName META_TABLE_NAME; + static { + Configuration conf = HBaseConfiguration.create(); + META_TABLE_NAME = initializeHbaseMetaTableName(conf); + LOG.info("Meta table name: {}", META_TABLE_NAME); + } + + /* Visible for testing only */ + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public static TableName getDefaultNameOfMetaForReplica() { + return valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + } + + public static TableName initializeHbaseMetaTableName(Configuration conf) { + String suffix_val = conf.get(HConstants.HBASE_META_TABLE_SUFFIX, + HConstants.HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE); + LOG.info("Meta table suffix value: {}", suffix_val); + if (Strings.isNullOrEmpty(suffix_val)) { + return valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + } else { + return valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta_" + suffix_val); + } + } /** * The Namespace table's name. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableForReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableForReplica.java new file mode 100644 index 000000000000..d4e3d2c8ce00 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableForReplica.java @@ -0,0 +1,114 @@ +/* + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test {@link org.apache.hadoop.hbase.TestMetaTableForReplica}. + */ +@Category({ MiscTests.class, MediumTests.class }) +@SuppressWarnings("deprecation") +public class TestMetaTableForReplica { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetaTableForReplica.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableForReplica.class); + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + private static Connection connection; + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void beforeClass() throws Exception { + Configuration c = UTIL.getConfiguration(); + // quicker heartbeat interval for faster DN death notification + c.setInt("hbase.ipc.client.connect.max.retries", 1); + c.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); + // Start cluster having non-default hbase meta table name + c.setStrings(HConstants.HBASE_META_TABLE_SUFFIX, "test"); + UTIL.startMiniCluster(3); + connection = ConnectionFactory.createConnection(c); + } + + @AfterClass + public static void afterClass() throws Exception { + connection.close(); + UTIL.shutdownMiniCluster(); + } + + @Test + public void testStateOfMetaForReplica() { + HMaster m = UTIL.getMiniHBaseCluster().getMaster(); + assertTrue(m.waitForMetaOnline()); + } + + @Test + public void testNameOfMetaForReplica() { + // Check the correctness of the meta table for replica + String metaTableName = TableName.META_TABLE_NAME.getNameWithNamespaceInclAsString(); + assertNotNull(metaTableName); + + // Check if name of the meta table for replica is not same as default table + assertEquals(0, + TableName.META_TABLE_NAME.compareTo(TableName.getDefaultNameOfMetaForReplica())); + } + + @Test + public void testGetNonExistentRegionFromMetaFromReplica() throws IOException { + final String name = this.name.getMethodName(); + LOG.info("Started " + name); + Pair pair = + MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region")); + assertNull(pair); + LOG.info("Finished " + name); + } + + @Test + public void testGetExistentRegionFromMetaFromReplica() throws IOException { + final TableName tableName = TableName.valueOf(name.getMethodName()); + LOG.info("Started " + tableName); + UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); + assertEquals(1, MetaTableAccessor.getTableRegions(connection, tableName).size()); + } +} From 2ed18d7cbb69b1cf0943d6bf734cbf495c410abd Mon Sep 17 00:00:00 2001 From: Anuj Sharma Date: Tue, 22 Apr 2025 21:58:45 +0530 Subject: [PATCH 02/35] HBASE-29083: Add global read-only mode to HBase (#6757) * HBASE-29083: Add global read-only mode to HBase Add hbase read-only property and ReadOnlyController (cherry picked from commit 49b678da90288bc645fcbfb8c0bbd27b33281c0f) * HBASE-29083. Allow test to update hbase:meta table * HBASE-29083. Spotless apply * Refactor code to have only passing tests * Apply spotless --------- Co-authored-by: Andor Molnar --- .../org/apache/hadoop/hbase/HConstants.java | 10 + .../security/access/ReadOnlyController.java | 393 ++++++++++++++++++ .../access/TestReadOnlyController.java | 100 +++++ 3 files changed, 503 insertions(+) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 4f24a739786b..e9ff37e2caa5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1677,6 +1677,16 @@ public enum OperationStatusCode { */ public final static String HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE = ""; + /** + * Should HBase only serve Read Requests + */ + public final static String HBASE_GLOBAL_READONLY_ENABLED_KEY = "hbase.global.readonly.enabled"; + + /** + * Default value of {@link #HBASE_GLOBAL_READONLY_ENABLED_KEY} + */ + public final static boolean HBASE_GLOBAL_READONLY_ENABLED_DEFAULT = false; + private HConstants() { // Can't be instantiated with this ctor. } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java new file mode 100644 index 000000000000..90d154ebec5b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java @@ -0,0 +1,393 @@ +/* + * 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.security.access; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +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.SnapshotDescription; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver; +import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; +import org.apache.hadoop.hbase.coprocessor.EndpointObserver; +import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; +import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.MasterObserver; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionServerObserver; +import org.apache.hadoop.hbase.filter.ByteArrayComparable; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.Service; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; + +@CoreCoprocessor +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class ReadOnlyController + implements MasterCoprocessor, RegionCoprocessor, MasterObserver, RegionObserver, + RegionServerCoprocessor, RegionServerObserver, EndpointObserver, BulkLoadObserver { + + private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyController.class); + private Configuration conf; + + private void internalReadOnlyGuard() throws IOException { + if ( + conf.getBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT) + ) { + // throw new FailedSanityCheckException("Operation not allowed in Read-Only Mode"); + throw new IOException("Operation not allowed in Read-Only Mode"); + } + } + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + conf = env.getConfiguration(); + } + + @Override + public void stop(CoprocessorEnvironment env) { + } + + /* ---- RegionObserver Overrides ---- */ + @Override + public Optional getRegionObserver() { + return Optional.of(this); + } + + @Override + public void prePut(ObserverContext c, Put put, + WALEdit edit) throws IOException { + if (edit.isMetaEdit() || edit.isEmpty()) { + return; + } + internalReadOnlyGuard(); + } + + @Override + public void preDelete(ObserverContext c, Delete delete, + WALEdit edit) throws IOException { + internalReadOnlyGuard(); + } + + @Override + public void preBatchMutate(ObserverContext c, + MiniBatchOperationInProgress miniBatchOp) throws IOException { + for (int i = 0; i < miniBatchOp.size(); i++) { + WALEdit edit = miniBatchOp.getWalEdit(i); + if (edit == null || edit.isMetaEdit() || edit.isEmpty()) { + continue; + } + internalReadOnlyGuard(); + } + } + + @Override + public void preFlush(final ObserverContext c, + FlushLifeCycleTracker tracker) throws IOException { + internalReadOnlyGuard(); + } + + @Override + public boolean preCheckAndPut(ObserverContext c, + byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, + Put put, boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndPut(c, row, family, qualifier, op, comparator, put, + result); + } + + @Override + public boolean preCheckAndPut(ObserverContext c, + byte[] row, Filter filter, Put put, boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndPut(c, row, filter, put, result); + } + + @Override + public boolean preCheckAndPutAfterRowLock( + ObserverContext c, byte[] row, byte[] family, + byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) + throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndPutAfterRowLock(c, row, family, qualifier, op, + comparator, put, result); + } + + @Override + public boolean preCheckAndPutAfterRowLock( + ObserverContext c, byte[] row, Filter filter, Put put, + boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndPutAfterRowLock(c, row, filter, put, result); + } + + @Override + public boolean preCheckAndDelete(ObserverContext c, + byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, + Delete delete, boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndDelete(c, row, family, qualifier, op, comparator, delete, + result); + } + + @Override + public boolean preCheckAndDelete(ObserverContext c, + byte[] row, Filter filter, Delete delete, boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndDelete(c, row, filter, delete, result); + } + + @Override + public boolean preCheckAndDeleteAfterRowLock( + ObserverContext c, byte[] row, byte[] family, + byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, + boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndDeleteAfterRowLock(c, row, family, qualifier, op, + comparator, delete, result); + } + + @Override + public boolean preCheckAndDeleteAfterRowLock( + ObserverContext c, byte[] row, Filter filter, + Delete delete, boolean result) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preCheckAndDeleteAfterRowLock(c, row, filter, delete, result); + } + + @Override + public Result preAppend(ObserverContext c, Append append) + throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preAppend(c, append); + } + + @Override + public Result preAppend(ObserverContext c, Append append, + WALEdit edit) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preAppend(c, append, edit); + } + + @Override + public Result preAppendAfterRowLock(ObserverContext c, + Append append) throws IOException { + internalReadOnlyGuard(); + return RegionObserver.super.preAppendAfterRowLock(c, append); + } + + @Override + public void preBulkLoadHFile(ObserverContext ctx, + List> familyPaths) throws IOException { + internalReadOnlyGuard(); + RegionObserver.super.preBulkLoadHFile(ctx, familyPaths); + } + + /* ---- MasterObserver Overrides ---- */ + @Override + public Optional getMasterObserver() { + return Optional.of(this); + } + + @Override + public void preCreateTable(ObserverContext ctx, + TableDescriptor desc, RegionInfo[] regions) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preCreateTable(ctx, desc, regions); + } + + @Override + public void preDeleteTable(ObserverContext ctx, TableName tableName) + throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preDeleteTable(ctx, tableName); + } + + @Override + public void preDeleteTableAction(ObserverContext ctx, + TableName tableName) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preDeleteTableAction(ctx, tableName); + } + + @Override + public void preTruncateTable(ObserverContext ctx, + TableName tableName) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preTruncateTable(ctx, tableName); + } + + @Override + public void preTruncateTableAction(ObserverContext ctx, + TableName tableName) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preTruncateTableAction(ctx, tableName); + } + + @Override + public TableDescriptor preModifyTable(ObserverContext ctx, + TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) + throws IOException { + internalReadOnlyGuard(); + return MasterObserver.super.preModifyTable(ctx, tableName, currentDescriptor, newDescriptor); + } + + @Override + public void preSnapshot(ObserverContext ctx, + SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preSnapshot(ctx, snapshot, tableDescriptor); + } + + @Override + public void preCloneSnapshot(ObserverContext ctx, + SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preCloneSnapshot(ctx, snapshot, tableDescriptor); + } + + @Override + public void preRestoreSnapshot(ObserverContext ctx, + SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preRestoreSnapshot(ctx, snapshot, tableDescriptor); + } + + @Override + public void preDeleteSnapshot(ObserverContext ctx, + SnapshotDescription snapshot) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preDeleteSnapshot(ctx, snapshot); + } + + @Override + public void preCreateNamespace(ObserverContext ctx, + NamespaceDescriptor ns) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preCreateNamespace(ctx, ns); + } + + @Override + public void preModifyNamespace(ObserverContext ctx, + NamespaceDescriptor currentNsDescriptor, NamespaceDescriptor newNsDescriptor) + throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preModifyNamespace(ctx, currentNsDescriptor, newNsDescriptor); + } + + @Override + public void preDeleteNamespace(ObserverContext ctx, + String namespace) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preDeleteNamespace(ctx, namespace); + } + + @Override + public void preMergeRegionsAction(ObserverContext ctx, + RegionInfo[] regionsToMerge) throws IOException { + internalReadOnlyGuard(); + MasterObserver.super.preMergeRegionsAction(ctx, regionsToMerge); + } + + /* ---- RegionServerObserver Overrides ---- */ + @Override + public void preRollWALWriterRequest(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + RegionServerObserver.super.preRollWALWriterRequest(ctx); + } + + @Override + public void preClearCompactionQueues(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + RegionServerObserver.super.preClearCompactionQueues(ctx); + } + + @Override + public void preExecuteProcedures(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + RegionServerObserver.super.preExecuteProcedures(ctx); + } + + @Override + public void preReplicationSinkBatchMutate(ObserverContext ctx, + AdminProtos.WALEntry walEntry, Mutation mutation) throws IOException { + internalReadOnlyGuard(); + RegionServerObserver.super.preReplicationSinkBatchMutate(ctx, walEntry, mutation); + } + + @Override + public void preClearRegionBlockCache(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + RegionServerObserver.super.preClearRegionBlockCache(ctx); + } + + /* ---- EndpointObserver Overrides ---- */ + @Override + public Message preEndpointInvocation(ObserverContext ctx, + Service service, String methodName, Message request) throws IOException { + internalReadOnlyGuard(); + return EndpointObserver.super.preEndpointInvocation(ctx, service, methodName, request); + } + + /* ---- BulkLoadObserver Overrides ---- */ + @Override + public void prePrepareBulkLoad(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + BulkLoadObserver.super.prePrepareBulkLoad(ctx); + } + + @Override + public void preCleanupBulkLoad(ObserverContext ctx) + throws IOException { + internalReadOnlyGuard(); + BulkLoadObserver.super.preCleanupBulkLoad(ctx); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java new file mode 100644 index 000000000000..1b286214e6d7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java @@ -0,0 +1,100 @@ +/* + * 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.security.access; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ SecurityTests.class, LargeTests.class }) +@SuppressWarnings("deprecation") +public class TestReadOnlyController { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReadOnlyController.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestAccessController.class); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static TableName TEST_TABLE = TableName.valueOf("readonlytesttable"); + private static byte[] TEST_FAMILY = Bytes.toBytes("readonlytablecolfam"); + private static Configuration conf; + private static Connection connection; + + private static RegionServerCoprocessorEnvironment RSCP_ENV; + + private static Table TestTable; + @Rule + public TestName name = new TestName(); + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @BeforeClass + public static void beforeClass() throws Exception { + conf = TEST_UTIL.getConfiguration(); + // Only try once so that if there is failure in connection then test should fail faster + conf.setInt("hbase.ipc.client.connect.max.retries", 1); + // Shorter session timeout is added so that in case failures test should not take more time + conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); + // Enable ReadOnly mode for the cluster + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, true); + // Add the ReadOnlyController coprocessor for region server to interrupt any write operation + conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ReadOnlyController.class.getName()); + // Add the ReadOnlyController coprocessor to for master to interrupt any write operation + conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ReadOnlyController.class.getName()); + // Start the test cluster + TEST_UTIL.startMiniCluster(2); + // Get connection to the HBase + connection = ConnectionFactory.createConnection(conf); + } + + @AfterClass + public static void afterClass() throws Exception { + if (connection != null) { + connection.close(); + } + TEST_UTIL.shutdownMiniCluster(); + } + + @Test(expected = IOException.class) + public void testCreateTable() throws IOException { + TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); + } +} From a8beb21e17fc651ccaf7f87083c361a77d1fc3da Mon Sep 17 00:00:00 2001 From: Kevin Geiszler Date: Thu, 29 May 2025 05:40:52 -0700 Subject: [PATCH 03/35] HBASE-29236: Add Support for Dynamic Configuration at the Coprocessor Level (#6931) Co-authored-by: Andor Molnar Co-authored-by: Anuj Sharma --- .../hbase/coprocessor/CoprocessorHost.java | 35 ++++ .../apache/hadoop/hbase/master/HMaster.java | 17 +- .../hadoop/hbase/regionserver/HRegion.java | 12 ++ .../hbase/regionserver/HRegionServer.java | 6 + .../security/access/ReadOnlyController.java | 44 +++-- .../TestCanStartHBaseInReadOnlyMode.java | 74 ++++++++ .../access/TestReadOnlyController.java | 170 +++++++++++++++--- 7 files changed, 315 insertions(+), 43 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCanStartHBaseInReadOnlyMode.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index 3d5897c0a056..625ac1dc5842 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.conf.ConfigurationManager; +import org.apache.hadoop.hbase.conf.ConfigurationObserver; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.CoprocessorClassLoader; @@ -130,6 +132,39 @@ public Set getCoprocessorClassNames() { return returnValue; } + /** + * Used to help make the relevant loaded coprocessors dynamically configurable by registering them + * to the {@link ConfigurationManager}. Coprocessors are considered "relevant" if they implement + * the {@link ConfigurationObserver} interface. + * @param configurationManager the ConfigurationManager the coprocessors get registered to + */ + public void registerConfigurationObservers(ConfigurationManager configurationManager) { + Coprocessor foundCp; + Set coprocessors = this.getCoprocessors(); + for (String cp : coprocessors) { + foundCp = this.findCoprocessor(cp); + if (foundCp instanceof ConfigurationObserver) { + configurationManager.registerObserver((ConfigurationObserver) foundCp); + } + } + } + + /** + * Deregisters relevant coprocessors from the {@link ConfigurationManager}. Coprocessors are + * considered "relevant" if they implement the {@link ConfigurationObserver} interface. + * @param configurationManager the ConfigurationManager the coprocessors get deregistered from + */ + public void deregisterConfigurationObservers(ConfigurationManager configurationManager) { + Coprocessor foundCp; + Set coprocessors = this.getCoprocessors(); + for (String cp : coprocessors) { + foundCp = this.findCoprocessor(cp); + if (foundCp instanceof ConfigurationObserver) { + configurationManager.deregisterObserver((ConfigurationObserver) foundCp); + } + } + } + /** * Load system coprocessors once only. Read the class names from configuration. Called by * constructor. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index a0f84c5672c3..b86644d5eda5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -611,6 +611,12 @@ protected String getUseThisHostnameInstead(Configuration conf) { private void registerConfigurationObservers() { configurationManager.registerObserver(this.rpcServices); configurationManager.registerObserver(this); + if (cpHost != null) { + cpHost.registerConfigurationObservers(configurationManager); + } else { + LOG.warn("Could not register HMaster coprocessors to the ConfigurationManager because " + + "MasterCoprocessorHost is null"); + } } // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will @@ -619,7 +625,6 @@ private void registerConfigurationObservers() { public void run() { try { installShutdownHook(); - registerConfigurationObservers(); Threads.setDaemonThreadRunning(new Thread(TraceUtil.tracedRunnable(() -> { try { int infoPort = putUpJettyServer(); @@ -1080,7 +1085,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE if (!maintenanceMode) { startupTaskGroup.addTask("Initializing master coprocessors"); setQuotasObserver(conf); - this.cpHost = new MasterCoprocessorHost(this, conf); + initializeCoprocessorHost(conf); } else { // start an in process region server for carrying system regions maintenanceRegionServer = @@ -4489,7 +4494,7 @@ public void onConfigurationChange(Configuration newConf) { CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY) && !maintenanceMode ) { LOG.info("Update the master coprocessor(s) because the configuration has changed"); - this.cpHost = new MasterCoprocessorHost(this, newConf); + initializeCoprocessorHost(newConf); } } @@ -4588,6 +4593,12 @@ private void setQuotasObserver(Configuration conf) { } } + private void initializeCoprocessorHost(Configuration conf) { + // initialize master side coprocessors before we start handling requests + this.cpHost = new MasterCoprocessorHost(this, conf); + registerConfigurationObservers(); + } + @Override public long flushTable(TableName tableName, List columnFamilies, long nonceGroup, long nonce) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index a2e38e532797..3bea489f2537 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -9001,6 +9001,12 @@ public void onConfigurationChange(Configuration conf) { public void registerChildren(ConfigurationManager manager) { configurationManager = manager; stores.values().forEach(manager::registerObserver); + if (coprocessorHost != null) { + coprocessorHost.registerConfigurationObservers(manager); + } else { + LOG.warn("Could not register HRegion coprocessors to the ConfigurationManager because " + + "RegionCoprocessorHost is null"); + } } /** @@ -9009,6 +9015,12 @@ public void registerChildren(ConfigurationManager manager) { @Override public void deregisterChildren(ConfigurationManager manager) { stores.values().forEach(configurationManager::deregisterObserver); + if (coprocessorHost != null) { + coprocessorHost.deregisterConfigurationObservers(manager); + } else { + LOG.warn("Could not deregister HRegion coprocessors from the ConfigurationManager because " + + "RegionCoprocessorHost is null"); + } } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index eea82ca511eb..6e6717d4238a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -2149,6 +2149,12 @@ private void registerConfigurationObservers() { configurationManager.registerObserver(this.rpcServices); configurationManager.registerObserver(this.prefetchExecutorNotifier); configurationManager.registerObserver(this); + if (rsHost != null) { + rsHost.registerConfigurationObservers(configurationManager); + } else { + LOG.warn("Could not register HRegionServer coprocessors to the ConfigurationManager because " + + "RegionServerCoprocessorHost is null"); + } } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java index 90d154ebec5b..13f458299b90 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.conf.ConfigurationObserver; import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver; import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; import org.apache.hadoop.hbase.coprocessor.EndpointObserver; @@ -65,26 +66,24 @@ @CoreCoprocessor @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) -public class ReadOnlyController - implements MasterCoprocessor, RegionCoprocessor, MasterObserver, RegionObserver, - RegionServerCoprocessor, RegionServerObserver, EndpointObserver, BulkLoadObserver { +public class ReadOnlyController implements MasterCoprocessor, RegionCoprocessor, MasterObserver, + RegionObserver, RegionServerCoprocessor, RegionServerObserver, EndpointObserver, BulkLoadObserver, + ConfigurationObserver { private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyController.class); - private Configuration conf; + private volatile boolean globalReadOnlyEnabled; private void internalReadOnlyGuard() throws IOException { - if ( - conf.getBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, - HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT) - ) { - // throw new FailedSanityCheckException("Operation not allowed in Read-Only Mode"); + if (this.globalReadOnlyEnabled) { throw new IOException("Operation not allowed in Read-Only Mode"); } } @Override public void start(CoprocessorEnvironment env) throws IOException { - conf = env.getConfiguration(); + this.globalReadOnlyEnabled = + env.getConfiguration().getBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT); } @Override @@ -100,7 +99,8 @@ public Optional getRegionObserver() { @Override public void prePut(ObserverContext c, Put put, WALEdit edit) throws IOException { - if (edit.isMetaEdit() || edit.isEmpty()) { + TableName tableName = c.getEnvironment().getRegionInfo().getTable(); + if (tableName.isSystemTable()) { return; } internalReadOnlyGuard(); @@ -115,13 +115,11 @@ public void preDelete(ObserverContext c, @Override public void preBatchMutate(ObserverContext c, MiniBatchOperationInProgress miniBatchOp) throws IOException { - for (int i = 0; i < miniBatchOp.size(); i++) { - WALEdit edit = miniBatchOp.getWalEdit(i); - if (edit == null || edit.isMetaEdit() || edit.isEmpty()) { - continue; - } - internalReadOnlyGuard(); + TableName tableName = c.getEnvironment().getRegionInfo().getTable(); + if (tableName.isSystemTable()) { + return; } + internalReadOnlyGuard(); } @Override @@ -390,4 +388,16 @@ public void preCleanupBulkLoad(ObserverContext ctx internalReadOnlyGuard(); BulkLoadObserver.super.preCleanupBulkLoad(ctx); } + + /* ---- ConfigurationObserver Overrides ---- */ + @Override + public void onConfigurationChange(Configuration conf) { + boolean maybeUpdatedConfValue = conf.getBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT); + if (this.globalReadOnlyEnabled != maybeUpdatedConfValue) { + this.globalReadOnlyEnabled = maybeUpdatedConfValue; + LOG.info("Config {} has been dynamically changed to {}", + HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, this.globalReadOnlyEnabled); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCanStartHBaseInReadOnlyMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCanStartHBaseInReadOnlyMode.java new file mode 100644 index 000000000000..ed182edbe91d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCanStartHBaseInReadOnlyMode.java @@ -0,0 +1,74 @@ +/* + * 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.security.access; + +import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_RETRIES_NUMBER; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.junit.*; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ SecurityTests.class, LargeTests.class }) +@SuppressWarnings("deprecation") +public class TestCanStartHBaseInReadOnlyMode { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCanStartHBaseInReadOnlyMode.class); + + private static final String READ_ONLY_CONTROLLER_NAME = ReadOnlyController.class.getName(); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static Configuration conf; + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void beforeClass() throws Exception { + conf = TEST_UTIL.getConfiguration(); + + // Shorten the run time of failed unit tests by limiting retries and the session timeout + // threshold + conf.setInt(HBASE_CLIENT_RETRIES_NUMBER, 0); + conf.setInt("hbase.master.init.timeout.localHBaseCluster", 10000); + + // Enable Read-Only mode to prove the cluster can be started in this state + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, true); + + // Add ReadOnlyController coprocessors to the master, region server, and regions + conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, READ_ONLY_CONTROLLER_NAME); + conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, READ_ONLY_CONTROLLER_NAME); + conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, READ_ONLY_CONTROLLER_NAME); + } + + @AfterClass + public static void afterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testCanStartHBaseInReadOnlyMode() throws Exception { + TEST_UTIL.startMiniCluster(1); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java index 1b286214e6d7..ddf513fcfe72 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestReadOnlyController.java @@ -17,17 +17,26 @@ */ package org.apache.hadoop.hbase.security.access; +import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_RETRIES_NUMBER; + import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; 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.Put; +import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; @@ -50,16 +59,18 @@ public class TestReadOnlyController { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestReadOnlyController.class); - private static final Logger LOG = LoggerFactory.getLogger(TestAccessController.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReadOnlyController.class); + private static final String READ_ONLY_CONTROLLER_NAME = ReadOnlyController.class.getName(); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); - private static TableName TEST_TABLE = TableName.valueOf("readonlytesttable"); - private static byte[] TEST_FAMILY = Bytes.toBytes("readonlytablecolfam"); + private static final TableName TEST_TABLE = TableName.valueOf("read_only_test_table"); + private static final byte[] TEST_FAMILY = Bytes.toBytes("read_only_table_col_fam"); + private static HRegionServer hRegionServer; + private static HMaster hMaster; private static Configuration conf; private static Connection connection; + private static SingleProcessHBaseCluster cluster; - private static RegionServerCoprocessorEnvironment RSCP_ENV; - - private static Table TestTable; + private static Table testTable; @Rule public TestName name = new TestName(); @@ -69,20 +80,39 @@ public class TestReadOnlyController { @BeforeClass public static void beforeClass() throws Exception { conf = TEST_UTIL.getConfiguration(); - // Only try once so that if there is failure in connection then test should fail faster - conf.setInt("hbase.ipc.client.connect.max.retries", 1); - // Shorter session timeout is added so that in case failures test should not take more time + + // Shorten the run time of failed unit tests by limiting retries and the session timeout + // threshold + conf.setInt(HBASE_CLIENT_RETRIES_NUMBER, 1); conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); - // Enable ReadOnly mode for the cluster - conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, true); - // Add the ReadOnlyController coprocessor for region server to interrupt any write operation - conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ReadOnlyController.class.getName()); - // Add the ReadOnlyController coprocessor to for master to interrupt any write operation - conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ReadOnlyController.class.getName()); - // Start the test cluster - TEST_UTIL.startMiniCluster(2); - // Get connection to the HBase - connection = ConnectionFactory.createConnection(conf); + + // Set up test class with Read-Only mode disabled so a table can be created + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, false); + + // Add ReadOnlyController coprocessors to the master, region server, and regions + conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, READ_ONLY_CONTROLLER_NAME); + conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, READ_ONLY_CONTROLLER_NAME); + conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, READ_ONLY_CONTROLLER_NAME); + + try { + // Start the test cluster + cluster = TEST_UTIL.startMiniCluster(1); + + hMaster = cluster.getMaster(); + hRegionServer = cluster.getRegionServerThreads().get(0).getRegionServer(); + connection = ConnectionFactory.createConnection(conf); + + // Create a test table + testTable = TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); + } catch (Exception e) { + // Delete the created table, and clean up the connection and cluster before throwing an + // exception + disableReadOnlyMode(); + TEST_UTIL.deleteTable(TEST_TABLE); + connection.close(); + TEST_UTIL.shutdownMiniCluster(); + throw new RuntimeException(e); + } } @AfterClass @@ -93,8 +123,102 @@ public static void afterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } - @Test(expected = IOException.class) - public void testCreateTable() throws IOException { - TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); + private static void enableReadOnlyMode() { + // Dynamically enable Read-Only mode if it is not active + if (!isReadOnlyModeEnabled()) { + LOG.info("Dynamically enabling Read-Only mode by setting {} to true", + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT); + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, true); + notifyObservers(); + } + } + + private static void disableReadOnlyMode() { + // Dynamically disable Read-Only mode if it is active + if (isReadOnlyModeEnabled()) { + LOG.info("Dynamically disabling Read-Only mode by setting {} to false", + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT); + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, false); + notifyObservers(); + } + } + + private static boolean isReadOnlyModeEnabled() { + return conf.getBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT); + } + + private static void notifyObservers() { + LOG.info("Notifying observers about configuration changes"); + hMaster.getConfigurationManager().notifyAllObservers(conf); + hRegionServer.getConfigurationManager().notifyAllObservers(conf); + } + + // The test case for successfully creating a table with Read-Only mode disabled happens when + // setting up the test class, so we only need a test function for a failed table creation. + @Test + public void testCannotCreateTableWithReadOnlyEnabled() throws IOException { + // Expect an IOException to result from the createTable attempt since Read-Only mode is enabled + enableReadOnlyMode(); + TableName newTable = TableName.valueOf("bad_read_only_test_table"); + exception.expect(IOException.class); + exception.expectMessage("Operation not allowed in Read-Only Mode"); + + // This should throw the IOException + TEST_UTIL.createTable(newTable, TEST_FAMILY); + } + + @Test + public void testPutWithReadOnlyDisabled() throws IOException { + // Successfully put a row in the table since Read-Only mode is disabled + disableReadOnlyMode(); + final byte[] row2 = Bytes.toBytes("row2"); + final byte[] value = Bytes.toBytes("efgh"); + Put put = new Put(row2); + put.addColumn(TEST_FAMILY, null, value); + testTable.put(put); + } + + @Test + public void testCannotPutWithReadOnlyEnabled() throws IOException { + // Prepare a Put command with Read-Only mode enabled + enableReadOnlyMode(); + final byte[] row1 = Bytes.toBytes("row1"); + final byte[] value = Bytes.toBytes("abcd"); + Put put = new Put(row1); + put.addColumn(TEST_FAMILY, null, value); + + // Expect an IOException to result from the Put attempt + exception.expect(IOException.class); + exception.expectMessage("Operation not allowed in Read-Only Mode"); + + // This should throw the IOException + testTable.put(put); + } + + @Test + public void testBatchPutWithReadOnlyDisabled() throws IOException, InterruptedException { + // Successfully create and run a batch Put operation with Read-Only mode disabled + disableReadOnlyMode(); + List actions = new ArrayList<>(); + actions.add(new Put(Bytes.toBytes("row10")).addColumn(TEST_FAMILY, null, Bytes.toBytes("10"))); + actions.add(new Delete(Bytes.toBytes("row10"))); + testTable.batch(actions, null); + } + + @Test + public void testCannotBatchPutWithReadOnlyEnabled() throws IOException, InterruptedException { + // Create a batch Put operation that is expected to fail with Read-Only mode enabled + enableReadOnlyMode(); + List actions = new ArrayList<>(); + actions.add(new Put(Bytes.toBytes("row11")).addColumn(TEST_FAMILY, null, Bytes.toBytes("11"))); + actions.add(new Delete(Bytes.toBytes("row11"))); + + // Expect an IOException to result from the batch Put attempt + exception.expect(IOException.class); + exception.expectMessage("Operation not allowed in Read-Only Mode"); + + // This should throw the IOException + testTable.batch(actions, null); } } From 0a7b45ce8116816f8edaf92a9a9e02509a7ec347 Mon Sep 17 00:00:00 2001 From: Abhishek Kothalikar <99398985+kabhishek4@users.noreply.github.com> Date: Tue, 3 Jun 2025 21:22:31 +0530 Subject: [PATCH 04/35] HBASE-29228 Add support to prevent running multiple active clusters (#6887) --- .../hadoop/hbase/ActiveClusterSuffix.java | 98 +++++++++++ .../org/apache/hadoop/hbase/HConstants.java | 3 + .../protobuf/server/ActiveClusterSuffix.proto | 33 ++++ .../hadoop/hbase/master/MasterFileSystem.java | 77 ++++++++- .../org/apache/hadoop/hbase/util/FSUtils.java | 93 +++++++++-- .../regionserver/TestActiveClusterSuffix.java | 158 ++++++++++++++++++ 6 files changed, 446 insertions(+), 16 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ActiveClusterSuffix.java create mode 100644 hbase-protocol-shaded/src/main/protobuf/server/ActiveClusterSuffix.proto create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestActiveClusterSuffix.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ActiveClusterSuffix.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ActiveClusterSuffix.java new file mode 100644 index 000000000000..68fd61a22534 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ActiveClusterSuffix.java @@ -0,0 +1,98 @@ +/* + * 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; + +import java.io.IOException; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ActiveClusterSuffixProtos; + +/** + * The suffix for this cluster. It is serialized to the filesystem and up into zookeeper. This is a + * container for the id. Also knows how to serialize and deserialize the cluster id. + */ +@InterfaceAudience.Private +public class ActiveClusterSuffix { + private final String active_cluster_suffix; + + /** + * New ActiveClusterSuffix. + */ + + public ActiveClusterSuffix(final String cs) { + this.active_cluster_suffix = cs; + } + + public String getActiveClusterSuffix() { + return active_cluster_suffix; + } + + /** Returns The active cluster suffix serialized using pb w/ pb magic prefix */ + public byte[] toByteArray() { + return ProtobufUtil.prependPBMagic(convert().toByteArray()); + } + + /** + * Parse the serialized representation of the {@link ActiveClusterSuffix} + * @param bytes A pb serialized {@link ActiveClusterSuffix} instance with pb magic prefix + * @return An instance of {@link ActiveClusterSuffix} made from bytes + * @see #toByteArray() + */ + public static ActiveClusterSuffix parseFrom(final byte[] bytes) throws DeserializationException { + if (ProtobufUtil.isPBMagicPrefix(bytes)) { + int pblen = ProtobufUtil.lengthOfPBMagic(); + ActiveClusterSuffixProtos.ActiveClusterSuffix.Builder builder = + ActiveClusterSuffixProtos.ActiveClusterSuffix.newBuilder(); + ActiveClusterSuffixProtos.ActiveClusterSuffix cs = null; + try { + ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); + cs = builder.build(); + } catch (IOException e) { + throw new DeserializationException(e); + } + return convert(cs); + } else { + // Presume it was written out this way, the old way. + return new ActiveClusterSuffix(Bytes.toString(bytes)); + } + } + + /** Returns A pb instance to represent this instance. */ + public ActiveClusterSuffixProtos.ActiveClusterSuffix convert() { + ActiveClusterSuffixProtos.ActiveClusterSuffix.Builder builder = + ActiveClusterSuffixProtos.ActiveClusterSuffix.newBuilder(); + return builder.setActiveClusterSuffix(this.active_cluster_suffix).build(); + } + + /** Returns A {@link ActiveClusterSuffix} made from the passed in cs */ + public static ActiveClusterSuffix + convert(final ActiveClusterSuffixProtos.ActiveClusterSuffix cs) { + return new ActiveClusterSuffix(cs.getActiveClusterSuffix()); + } + + /** + * @see java.lang.Object#toString() + */ + @Override + public String toString() { + return this.active_cluster_suffix; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index e9ff37e2caa5..8020acb20b77 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1687,6 +1687,9 @@ public enum OperationStatusCode { */ public final static boolean HBASE_GLOBAL_READONLY_ENABLED_DEFAULT = false; + /** name of the file having active cluster suffix */ + public static final String ACTIVE_CLUSTER_SUFFIX_FILE_NAME = " active.cluster.suffix.id"; + private HConstants() { // Can't be instantiated with this ctor. } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/ActiveClusterSuffix.proto b/hbase-protocol-shaded/src/main/protobuf/server/ActiveClusterSuffix.proto new file mode 100644 index 000000000000..89bc086212b3 --- /dev/null +++ b/hbase-protocol-shaded/src/main/protobuf/server/ActiveClusterSuffix.proto @@ -0,0 +1,33 @@ +/** + * 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. + */ +syntax = "proto2"; +// This file contains protocol buffers that are shared throughout HBase +package hbase.pb; + +option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated"; +option java_outer_classname = "ActiveClusterSuffixProtos"; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +/** + * Content of the '/hbase/active_cluster_suffix.id' file to indicate the active cluster. + */ +message ActiveClusterSuffix { + // This is the active cluster suffix set by the user in the config, as a String + required string active_cluster_suffix = 1; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index 5a43cd98feb9..034faa058027 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java @@ -19,12 +19,15 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Objects; 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.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.ActiveClusterSuffix; import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.backup.HFileArchiver; @@ -60,6 +63,8 @@ public class MasterFileSystem { private final Configuration conf; // Persisted unique cluster ID private ClusterId clusterId; + // Persisted unique Active Cluster Suffix + private ActiveClusterSuffix activeClusterSuffix; // Keep around for convenience. private final FileSystem fs; // Keep around for convenience. @@ -158,6 +163,8 @@ private void createInitialFileSystemLayout() throws IOException { if (isSecurityEnabled) { fs.setPermission(new Path(rootdir, HConstants.VERSION_FILE_NAME), secureRootFilePerms); fs.setPermission(new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME), secureRootFilePerms); + fs.setPermission(new Path(rootdir, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME), + secureRootFilePerms); } FsPermission currentRootPerms = fs.getFileStatus(this.rootdir).getPermission(); if ( @@ -262,10 +269,14 @@ private void checkRootDir(final Path rd, final Configuration c, final FileSystem throw iae; } // Make sure cluster ID exists - if (!FSUtils.checkClusterIdExists(fs, rd, threadWakeFrequency)) { - FSUtils.setClusterId(fs, rd, new ClusterId(), threadWakeFrequency); + if ( + !FSUtils.checkFileExistsInHbaseRootDir(fs, rootdir, HConstants.CLUSTER_ID_FILE_NAME, + threadWakeFrequency) + ) { + FSUtils.setClusterId(fs, rootdir, new ClusterId(), threadWakeFrequency); } - clusterId = FSUtils.getClusterId(fs, rd); + clusterId = FSUtils.getClusterId(fs, rootdir); + negotiateActiveClusterSuffixFile(threadWakeFrequency); } /** @@ -382,4 +393,64 @@ public void stop() { public void logFileSystemState(Logger log) throws IOException { CommonFSUtils.logFileSystemState(fs, rootdir, log); } + + private void negotiateActiveClusterSuffixFile(long wait) throws IOException { + if (!isReadOnlyModeEnabled(conf)) { + try { + // verify the contents against the config set + ActiveClusterSuffix acs = FSUtils.getActiveClusterSuffix(fs, rootdir); + LOG.debug("File Suffix {} : Configured suffix {} : Cluster ID : {}", acs, + getSuffixFromConfig(), getClusterId()); + if (Objects.equals(acs.getActiveClusterSuffix(), getSuffixFromConfig())) { + this.activeClusterSuffix = acs; + } else { + // throw error + LOG.info("rootdir {} : Active Cluster File Suffix {} ", rootdir, acs); + throw new IOException("Cannot start master, because another cluster is running in active " + + "(read-write) mode on this storage location. Active Cluster Id: {} " + acs + + " This cluster Id: " + getClusterId()); + } + LOG.info( + "This is the active cluster on this storage location, " + "File Suffix {} : Suffix {} : ", + acs, getActiveClusterSuffix()); + } catch (FileNotFoundException fnfe) { + // this is the active cluster, create active cluster suffix file if it does not exist + FSUtils.setActiveClusterSuffix(fs, rootdir, getSuffixFileDataToWrite(), wait); + } + } else { + // this is a replica cluster + LOG.info("Replica cluster is being started in Read Only Mode"); + } + } + + public ActiveClusterSuffix getActiveClusterSuffix() { + return activeClusterSuffix; + } + + private boolean isReadOnlyModeEnabled(Configuration conf) { + return conf.getBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, + HConstants.HBASE_GLOBAL_READONLY_ENABLED_DEFAULT); + } + + private String getActiveClusterSuffixFromConfig(Configuration conf) { + return conf.get(HConstants.HBASE_META_TABLE_SUFFIX, + HConstants.HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE); + } + + public String getSuffixFromConfig() { + return getClusterId().toString() + ":" + getActiveClusterSuffixFromConfig(conf); + } + + // Used only for testing + public byte[] getSuffixFileDataToCompare() { + String str = this.activeClusterSuffix.toString(); + return str.getBytes(StandardCharsets.UTF_8); + } + + // + public byte[] getSuffixFileDataToWrite() { + String str = getClusterId().toString() + ":" + getActiveClusterSuffixFromConfig(conf); + this.activeClusterSuffix = new ActiveClusterSuffix(str); + return str.getBytes(StandardCharsets.UTF_8); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index d504571ea900..7b465b773162 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -31,6 +31,7 @@ import java.lang.reflect.Method; import java.net.InetSocketAddress; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -65,6 +66,7 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.ActiveClusterSuffix; import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -541,15 +543,15 @@ public static void setVersion(FileSystem fs, Path rootdir, String version, int w * @return true if the file exists, otherwise false * @throws IOException if checking the FileSystem fails */ - public static boolean checkClusterIdExists(FileSystem fs, Path rootdir, long wait) - throws IOException { + public static boolean checkFileExistsInHbaseRootDir(FileSystem fs, Path rootdir, String file, + long wait) throws IOException { while (true) { try { - Path filePath = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME); + Path filePath = new Path(rootdir, file); return fs.exists(filePath); } catch (IOException ioe) { if (wait > 0L) { - LOG.warn("Unable to check cluster ID file in {}, retrying in {}ms", rootdir, wait, ioe); + LOG.warn("Unable to check file {} in {}, retrying in {}ms", file, rootdir, wait, ioe); try { Thread.sleep(wait); } catch (InterruptedException e) { @@ -611,6 +613,46 @@ public static ClusterId getClusterId(FileSystem fs, Path rootdir) throws IOExcep return clusterId; } + public static ActiveClusterSuffix getActiveClusterSuffix(FileSystem fs, Path rootdir) + throws IOException { + Path idPath = new Path(rootdir, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME); + ActiveClusterSuffix cs = null; + FileStatus status = fs.exists(idPath) ? fs.getFileStatus(idPath) : null; + if (status != null) { + int len = Ints.checkedCast(status.getLen()); + byte[] content = new byte[len]; + FSDataInputStream in = fs.open(idPath); + try { + in.readFully(content); + } catch (EOFException eof) { + LOG.warn("Cluster Suffix file {} is empty ", idPath); + } finally { + in.close(); + } + try { + cs = ActiveClusterSuffix.parseFrom(content); + } catch (DeserializationException e) { + throw new IOException("content=" + Bytes.toString(content), e); + } + // If not pb'd, make it so. + if (!ProtobufUtil.isPBMagicPrefix(content)) { + String data = null; + in = fs.open(idPath); + try { + data = in.readUTF(); + cs = new ActiveClusterSuffix(data); + } catch (EOFException eof) { + LOG.warn("Active Cluster Suffix File {} is empty ", idPath); + } finally { + in.close(); + } + } + return cs; + } else { + throw new FileNotFoundException("Active Cluster Suffix File " + idPath + " not found"); + } + } + /** * */ private static void rewriteAsPb(final FileSystem fs, final Path rootdir, final Path p, @@ -638,31 +680,57 @@ private static void rewriteAsPb(final FileSystem fs, final Path rootdir, final P */ public static void setClusterId(final FileSystem fs, final Path rootdir, final ClusterId clusterId, final long wait) throws IOException { - final Path idFile = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME); final Path tempDir = new Path(rootdir, HConstants.HBASE_TEMP_DIRECTORY); final Path tempIdFile = new Path(tempDir, HConstants.CLUSTER_ID_FILE_NAME); LOG.debug("Create cluster ID file [{}] with ID: {}", idFile, clusterId); + writeClusterInfo(fs, rootdir, idFile, tempIdFile, clusterId.toByteArray(), wait); + } + + /** + * Writes a user provided suffix for this cluster to the "active_cluster_suffix.id" file in the + * HBase root directory. If any operations on the ID file fails, and {@code wait} is a positive + * value, the method will retry to produce the ID file until the thread is forcibly interrupted. + */ + + public static void setActiveClusterSuffix(final FileSystem fs, final Path rootdir, byte[] bdata, + final long wait) throws IOException { + final Path idFile = new Path(rootdir, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME); + final Path tempDir = new Path(rootdir, HConstants.HBASE_TEMP_DIRECTORY); + final Path tempIdFile = new Path(tempDir, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME); + String fsuffix = new String(bdata, StandardCharsets.US_ASCII); + + LOG.debug("Create Active cluster Suffix file [{}] with Suffix: {}", idFile, fsuffix); + writeClusterInfo(fs, rootdir, idFile, tempIdFile, bdata, wait); + } + + /** + * Writes information about this cluster to the specified file. For ex, it is used for writing + * cluster id in "hbase.id" file in the HBase root directory. Also, used for writing active + * cluster suffix in "active_cluster_suffix.id" file. If any operations on the ID file fails, and + * {@code wait} is a positive value, the method will retry to produce the ID file until the thread + * is forcibly interrupted. + */ + private static void writeClusterInfo(final FileSystem fs, final Path rootdir, final Path idFile, + final Path tempIdFile, byte[] fileData, final long wait) throws IOException { while (true) { Optional failure = Optional.empty(); - LOG.debug("Write the cluster ID file to a temporary location: {}", tempIdFile); + LOG.debug("Write the file to a temporary location: {}", tempIdFile); try (FSDataOutputStream s = fs.create(tempIdFile)) { - s.write(clusterId.toByteArray()); + s.write(fileData); } catch (IOException ioe) { failure = Optional.of(ioe); } if (!failure.isPresent()) { try { - LOG.debug("Move the temporary cluster ID file to its target location [{}]:[{}]", - tempIdFile, idFile); + LOG.debug("Move the temporary file to its target location [{}]:[{}]", tempIdFile, idFile); if (!fs.rename(tempIdFile, idFile)) { - failure = - Optional.of(new IOException("Unable to move temp cluster ID file to " + idFile)); + failure = Optional.of(new IOException("Unable to move temp file to " + idFile)); } } catch (IOException ioe) { failure = Optional.of(ioe); @@ -672,8 +740,7 @@ public static void setClusterId(final FileSystem fs, final Path rootdir, if (failure.isPresent()) { final IOException cause = failure.get(); if (wait > 0L) { - LOG.warn("Unable to create cluster ID file in {}, retrying in {}ms", rootdir, wait, - cause); + LOG.warn("Unable to create file in {}, retrying in {}ms", rootdir, wait, cause); try { Thread.sleep(wait); } catch (InterruptedException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestActiveClusterSuffix.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestActiveClusterSuffix.java new file mode 100644 index 000000000000..df036f08f005 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestActiveClusterSuffix.java @@ -0,0 +1,158 @@ +/* + * 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.io.IOException; +import java.nio.charset.StandardCharsets; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtil; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test Active Cluster Suffix file. + */ +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestActiveClusterSuffix { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestActiveClusterSuffix.class); + + private final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + private JVMClusterUtil.RegionServerThread rst; + + @Before + public void setUp() throws Exception { + TEST_UTIL.getConfiguration().setBoolean(ShutdownHook.RUN_SHUTDOWN_HOOK, false); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + if (rst != null && rst.getRegionServer() != null) { + rst.getRegionServer().stop("end of test"); + rst.join(); + } + } + + @Test + public void testActiveClusterSuffixCreated() throws Exception { + TEST_UTIL.startMiniZKCluster(); + TEST_UTIL.startMiniDFSCluster(1); + TEST_UTIL.startMiniHBaseCluster(); + + Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()); + FileSystem fs = rootDir.getFileSystem(TEST_UTIL.getConfiguration()); + Path filePath = new Path(rootDir, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME); + + assertTrue(filePath + " should exists ", fs.exists(filePath)); + assertTrue(filePath + " should not be empty ", fs.getFileStatus(filePath).getLen() > 0); + + MasterFileSystem mfs = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem(); + // Compute string using currently set suffix and the cluster id + String cluster_suffix1 = + new String(mfs.getSuffixFileDataToCompare(), StandardCharsets.US_ASCII); + // Compute string member variable + String cluster_suffix2 = mfs.getActiveClusterSuffix().toString(); + assertEquals(cluster_suffix1, cluster_suffix2); + } + + @Test + public void testSuffixFileOnRestart() throws Exception { + TEST_UTIL.startMiniZKCluster(); + TEST_UTIL.startMiniDFSCluster(1); + TEST_UTIL.createRootDir(); + TEST_UTIL.getConfiguration().set(HConstants.HBASE_META_TABLE_SUFFIX, "Test"); + + String clusterId = HBaseCommonTestingUtil.getRandomUUID().toString(); + String cluster_suffix = clusterId + ":" + TEST_UTIL.getConfiguration() + .get(HConstants.HBASE_META_TABLE_SUFFIX, HConstants.HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE); + + writeIdFile(clusterId, HConstants.CLUSTER_ID_FILE_NAME); + writeIdFile(cluster_suffix, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME); + + try { + TEST_UTIL.startMiniHBaseCluster(); + } catch (IOException ioe) { + Assert.fail("Can't start mini hbase cluster."); + } + + MasterFileSystem mfs = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem(); + // Compute using file contents + String cluster_suffix1 = + new String(mfs.getSuffixFileDataToCompare(), StandardCharsets.US_ASCII); + // Compute using config + String cluster_suffix2 = mfs.getSuffixFromConfig(); + + assertEquals(cluster_suffix1, cluster_suffix2); + assertEquals(cluster_suffix, cluster_suffix1); + } + + @Test + public void testVerifyErrorWhenSuffixNotMatched() throws Exception { + TEST_UTIL.startMiniZKCluster(); + TEST_UTIL.startMiniDFSCluster(1); + TEST_UTIL.createRootDir(); + TEST_UTIL.getConfiguration().setInt("hbase.master.start.timeout.localHBaseCluster", 10000); + String cluster_suffix = String.valueOf("2df92f65-d801-46e6-b892-c2bae2df3c21:test"); + writeIdFile(cluster_suffix, HConstants.ACTIVE_CLUSTER_SUFFIX_FILE_NAME); + // Exception: as config in the file and the one set by the user are not matching + boolean threwIOE = false; + try { + TEST_UTIL.startMiniHBaseCluster(); + } catch (IOException ioe) { + threwIOE = true; + } finally { + assertTrue("The master should have thrown an exception", threwIOE); + } + } + + private void writeIdFile(String id, String fileName) throws Exception { + Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()); + FileSystem fs = rootDir.getFileSystem(TEST_UTIL.getConfiguration()); + Path filePath = new Path(rootDir, fileName); + FSDataOutputStream s = null; + try { + s = fs.create(filePath); + s.writeUTF(id); + } finally { + if (s != null) { + s.close(); + } + } + } +} From 8e4f705f4c9852e4de5648262456597fb39ae920 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Thu, 11 Sep 2025 09:22:19 -0500 Subject: [PATCH 05/35] HBASE-29291: Add a command to refresh/sync hbase:meta table (#7058) Change-Id: Ia04bb12cdaf580f26cb14d9a34b5963105065faa --- .../org/apache/hadoop/hbase/client/Admin.java | 5 + .../hbase/client/AdminOverAsyncAdmin.java | 5 + .../hadoop/hbase/client/AsyncAdmin.java | 5 + .../hadoop/hbase/client/AsyncHBaseAdmin.java | 5 + .../hbase/client/RawAsyncHBaseAdmin.java | 13 + .../main/protobuf/server/master/Master.proto | 11 + .../server/master/MasterProcedure.proto | 12 + .../hadoop/hbase/MetaTableAccessor.java | 2 +- .../apache/hadoop/hbase/TableDescriptors.java | 7 + .../apache/hadoop/hbase/master/HMaster.java | 16 + .../hbase/master/MasterRpcServices.java | 11 + .../procedure/RefreshMetaProcedure.java | 480 ++++++++++++++++++ .../security/access/ReadOnlyController.java | 19 +- .../hadoop/hbase/util/FSTableDescriptors.java | 10 + .../procedure/TestRefreshMetaProcedure.java | 121 +++++ .../TestRefreshMetaProcedureIntegration.java | 285 +++++++++++ .../hbase/rsgroup/VerifyingRSGroupAdmin.java | 4 + hbase-shell/src/main/ruby/hbase/admin.rb | 6 + hbase-shell/src/main/ruby/shell.rb | 1 + .../main/ruby/shell/commands/refresh_meta.rb | 43 ++ .../hbase/thrift2/client/ThriftAdmin.java | 5 + 21 files changed, 1061 insertions(+), 5 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshMetaProcedure.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedure.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedureIntegration.java create mode 100644 hbase-shell/src/main/ruby/shell/commands/refresh_meta.rb diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 65b3abcd413c..8622e1c88774 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -2703,6 +2703,11 @@ List getLogEntries(Set serverNames, String logType, Server */ List getCachedFilesList(ServerName serverName) throws IOException; + /** + * Perform hbase:meta table refresh + */ + Long refreshMeta() throws IOException; + @InterfaceAudience.Private void restoreBackupSystemTable(String snapshotName) throws IOException; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java index 7117fd4fd33f..37d46feb41ad 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java @@ -1153,6 +1153,11 @@ public List getCachedFilesList(ServerName serverName) throws IOException return get(admin.getCachedFilesList(serverName)); } + @Override + public Long refreshMeta() throws IOException { + return get(admin.refreshMeta()); + } + @Override public void restoreBackupSystemTable(String snapshotName) throws IOException { get(admin.restoreBackupSystemTable(snapshotName)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index 56211cedc493..e1d3aadfed34 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -1890,6 +1890,11 @@ CompletableFuture> getLogEntries(Set serverNames, Str */ CompletableFuture> getCachedFilesList(ServerName serverName); + /** + * Perform hbase:meta table refresh + */ + CompletableFuture refreshMeta(); + @InterfaceAudience.Private CompletableFuture restoreBackupSystemTable(String snapshotName); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 8132b184809c..26abe68402ca 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -1021,6 +1021,11 @@ public CompletableFuture> getCachedFilesList(ServerName serverName) return wrap(rawAdmin.getCachedFilesList(serverName)); } + @Override + public CompletableFuture refreshMeta() { + return wrap(rawAdmin.refreshMeta()); + } + @Override public CompletableFuture restoreBackupSystemTable(String snapshotName) { return wrap(rawAdmin.restoreBackupSystemTable(snapshotName)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index ea51d27b99a4..9454d2826533 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -263,6 +263,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RefreshMetaRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RefreshMetaResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ReopenTableRegionsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ReopenTableRegionsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest; @@ -4697,4 +4699,15 @@ MasterProtos.RestoreBackupSystemTableResponse> procedureCall(request, MasterProtos.RestoreBackupSystemTableResponse::getProcId, new RestoreBackupSystemTableProcedureBiConsumer()); } + + @Override + public CompletableFuture refreshMeta() { + RefreshMetaRequest.Builder request = RefreshMetaRequest.newBuilder(); + request.setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()); + return this. newMasterCaller() + .action((controller, stub) -> this. call( + controller, stub, request.build(), MasterService.Interface::refreshMeta, + RefreshMetaResponse::getProcId)) + .call(); + } } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto index f475d26060d0..40f6a1518f13 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto @@ -819,6 +819,14 @@ message RollAllWALWritersResponse { optional uint64 proc_id = 1; } +message RefreshMetaRequest { + optional uint64 nonce_group = 1 [default = 0]; + optional uint64 nonce = 2 [default = 0]; +} +message RefreshMetaResponse { + optional uint64 proc_id = 1; +} + service MasterService { /** Used by the client to get the number of regions that have received the updated schema */ rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest) @@ -1303,6 +1311,9 @@ service MasterService { rpc rollAllWALWriters(RollAllWALWritersRequest) returns(RollAllWALWritersResponse); + + rpc RefreshMeta(RefreshMetaRequest) + returns(RefreshMetaResponse); } // HBCK Service definitions. diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto index 7e6c6c8e2fc7..aa79ff474c35 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto @@ -864,3 +864,15 @@ message LogRollRemoteProcedureResult { optional ServerName server_name = 1; optional uint64 last_highest_wal_filenum = 2; } + +enum RefreshMetaState { + REFRESH_META_INIT = 1; + REFRESH_META_SCAN_STORAGE = 2; + REFRESH_META_PREPARE = 3; + REFRESH_META_APPLY = 4; + REFRESH_META_FOLLOWUP = 5; + REFRESH_META_FINISH = 6; +} + +message RefreshMetaStateData { +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 05b049e27dbc..38aa054fd7c3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -753,7 +753,7 @@ private static void deleteFromMetaTable(final Connection connection, final Delet * @param connection connection we're using * @param deletes Deletes to add to hbase:meta This list should support #remove. */ - private static void deleteFromMetaTable(final Connection connection, final List deletes) + public static void deleteFromMetaTable(final Connection connection, final List deletes) throws IOException { try (Table t = getMetaHTable(connection)) { debugLogMutations(deletes); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java index d22e46383d30..32594ffce489 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java @@ -78,4 +78,11 @@ default void update(TableDescriptor htd) throws IOException { /** Returns Instance of table descriptor or null if none found. */ TableDescriptor remove(TableName tablename) throws IOException; + + /** + * Invalidates the table descriptor cache. + */ + default void invalidateTableDescriptorCache() { + // do nothing by default + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index b86644d5eda5..caca0fcef73f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -170,6 +170,7 @@ import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait; import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher; +import org.apache.hadoop.hbase.master.procedure.RefreshMetaProcedure; import org.apache.hadoop.hbase.master.procedure.ReloadQuotasProcedure; import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure; import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure; @@ -4652,4 +4653,19 @@ public MobFileCleanerChore getMobFileCleanerChore() { return mobFileCleanerChore; } + public Long refreshMeta(long nonceGroup, long nonce) throws IOException { + return MasterProcedureUtil + .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + @Override + protected void run() throws IOException { + LOG.info("Submitting RefreshMetaProcedure"); + submitProcedure(new RefreshMetaProcedure(procedureExecutor.getEnvironment())); + } + + @Override + protected String getDescription() { + return "RefreshMetaProcedure"; + } + }); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 85336eed2fb5..472a8f863982 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -3742,4 +3742,15 @@ public RollAllWALWritersResponse rollAllWALWriters(RpcController rpcController, throw new ServiceException(ioe); } } + + @Override + public MasterProtos.RefreshMetaResponse refreshMeta(RpcController controller, + MasterProtos.RefreshMetaRequest request) throws ServiceException { + try { + Long procId = server.refreshMeta(request.getNonceGroup(), request.getNonce()); + return MasterProtos.RefreshMetaResponse.newBuilder().setProcId(procId).build(); + } catch (IOException ioe) { + throw new ServiceException(ioe); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshMetaProcedure.java new file mode 100644 index 000000000000..b2e458cd495e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshMetaProcedure.java @@ -0,0 +1,480 @@ +/* + * 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.master.procedure; + +import static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState.WAITING_TIMEOUT; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureUtil; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.RetryCounter; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshMetaState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshMetaStateData; + +@InterfaceAudience.Private +public class RefreshMetaProcedure extends AbstractStateMachineTableProcedure { + private static final Logger LOG = LoggerFactory.getLogger(RefreshMetaProcedure.class); + private static final String HIDDEN_DIR_PATTERN = "^[._-].*"; + + private List currentRegions; + private List latestRegions; + private List pendingMutations; + private RetryCounter retryCounter; + private static final int MUTATION_BATCH_SIZE = 100; + private List newlyAddedRegions; + private List deletedTables; + + public RefreshMetaProcedure() { + super(); + } + + public RefreshMetaProcedure(MasterProcedureEnv env) { + super(env); + } + + @Override + public TableName getTableName() { + return TableName.META_TABLE_NAME; + } + + @Override + public TableOperationType getTableOperationType() { + return TableOperationType.EDIT; + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, RefreshMetaState refreshMetaState) { + LOG.info("Executing RefreshMetaProcedure state: {}", refreshMetaState); + + try { + return switch (refreshMetaState) { + case REFRESH_META_INIT -> executeInit(env); + case REFRESH_META_SCAN_STORAGE -> executeScanStorage(env); + case REFRESH_META_PREPARE -> executePrepare(); + case REFRESH_META_APPLY -> executeApply(env); + case REFRESH_META_FOLLOWUP -> executeFollowup(env); + case REFRESH_META_FINISH -> executeFinish(env); + default -> throw new UnsupportedOperationException("Unhandled state: " + refreshMetaState); + }; + } catch (Exception ex) { + LOG.error("Error in RefreshMetaProcedure state {}", refreshMetaState, ex); + setFailure("RefreshMetaProcedure", ex); + return Flow.NO_MORE_STATE; + } + } + + private Flow executeInit(MasterProcedureEnv env) throws IOException { + LOG.trace("Getting current regions from hbase:meta table"); + try { + currentRegions = getCurrentRegions(env.getMasterServices().getConnection()); + LOG.info("Found {} current regions in meta table", currentRegions.size()); + setNextState(RefreshMetaState.REFRESH_META_SCAN_STORAGE); + return Flow.HAS_MORE_STATE; + } catch (IOException ioe) { + LOG.error("Failed to get current regions from meta table", ioe); + throw ioe; + } + } + + private Flow executeScanStorage(MasterProcedureEnv env) throws IOException { + try { + latestRegions = scanBackingStorage(env.getMasterServices().getConnection()); + LOG.info("Found {} regions in backing storage", latestRegions.size()); + setNextState(RefreshMetaState.REFRESH_META_PREPARE); + return Flow.HAS_MORE_STATE; + } catch (IOException ioe) { + LOG.error("Failed to scan backing storage", ioe); + throw ioe; + } + } + + private Flow executePrepare() throws IOException { + if (currentRegions == null || latestRegions == null) { + LOG.error( + "Can not execute update on null lists. " + "Meta Table Regions - {}, Storage Regions - {}", + currentRegions, latestRegions); + throw new IOException( + (currentRegions == null ? "current regions" : "latest regions") + " list is null"); + } + LOG.info("Comparing regions. Current regions: {}, Latest regions: {}", currentRegions.size(), + latestRegions.size()); + + this.newlyAddedRegions = new ArrayList<>(); + this.deletedTables = new ArrayList<>(); + + pendingMutations = prepareMutations( + currentRegions.stream() + .collect(Collectors.toMap(RegionInfo::getEncodedName, Function.identity())), + latestRegions.stream() + .collect(Collectors.toMap(RegionInfo::getEncodedName, Function.identity()))); + + if (pendingMutations.isEmpty()) { + LOG.info("RefreshMetaProcedure completed, No update needed."); + setNextState(RefreshMetaState.REFRESH_META_FINISH); + } else { + LOG.info("Prepared {} region mutations and {} tables for cleanup.", pendingMutations.size(), + deletedTables.size()); + setNextState(RefreshMetaState.REFRESH_META_APPLY); + } + return Flow.HAS_MORE_STATE; + } + + private Flow executeApply(MasterProcedureEnv env) throws ProcedureSuspendedException { + try { + if (pendingMutations != null && !pendingMutations.isEmpty()) { + applyMutations(env.getMasterServices().getConnection(), pendingMutations); + LOG.debug("RefreshMetaProcedure applied {} mutations to meta table", + pendingMutations.size()); + } + } catch (IOException ioe) { + if (retryCounter == null) { + retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration()); + } + long backoff = retryCounter.getBackoffTimeAndIncrementAttempts(); + LOG.warn("Failed to apply mutations to meta table, suspending for {} ms", backoff, ioe); + setTimeout(Math.toIntExact(backoff)); + setState(WAITING_TIMEOUT); + skipPersistence(); + throw new ProcedureSuspendedException(); + } + + if ( + (this.newlyAddedRegions != null && !this.newlyAddedRegions.isEmpty()) + || (this.deletedTables != null && !this.deletedTables.isEmpty()) + ) { + setNextState(RefreshMetaState.REFRESH_META_FOLLOWUP); + } else { + LOG.info("RefreshMetaProcedure completed. No follow-up actions were required."); + setNextState(RefreshMetaState.REFRESH_META_FINISH); + } + return Flow.HAS_MORE_STATE; + } + + private Flow executeFollowup(MasterProcedureEnv env) throws IOException { + + LOG.info("Submitting assignment for new regions: {}", this.newlyAddedRegions); + addChildProcedure(env.getAssignmentManager().createAssignProcedures(newlyAddedRegions)); + + for (TableName tableName : this.deletedTables) { + LOG.debug("Submitting deletion for empty table {}", tableName); + env.getMasterServices().getAssignmentManager().deleteTable(tableName); + env.getMasterServices().getTableStateManager().setDeletedTable(tableName); + env.getMasterServices().getTableDescriptors().remove(tableName); + } + setNextState(RefreshMetaState.REFRESH_META_FINISH); + return Flow.HAS_MORE_STATE; + } + + private Flow executeFinish(MasterProcedureEnv env) { + invalidateTableDescriptorCache(env); + LOG.info("RefreshMetaProcedure completed successfully. All follow-up actions finished."); + currentRegions = null; + latestRegions = null; + pendingMutations = null; + deletedTables = null; + newlyAddedRegions = null; + return Flow.NO_MORE_STATE; + } + + private void invalidateTableDescriptorCache(MasterProcedureEnv env) { + LOG.debug("Invalidating the table descriptor cache to ensure new tables are discovered"); + env.getMasterServices().getTableDescriptors().invalidateTableDescriptorCache(); + } + + /** + * Prepares mutations by comparing the current regions in hbase:meta with the latest regions from + * backing storage. Also populates newlyAddedRegions and deletedTables lists for follow-up + * actions. + * @param currentMap Current regions from hbase:meta + * @param latestMap Latest regions from backing storage + * @return List of mutations to apply to the meta table + * @throws IOException If there is an error creating mutations + */ + private List prepareMutations(Map currentMap, + Map latestMap) throws IOException { + List mutations = new ArrayList<>(); + + for (String regionId : Stream.concat(currentMap.keySet().stream(), latestMap.keySet().stream()) + .collect(Collectors.toSet())) { + RegionInfo currentRegion = currentMap.get(regionId); + RegionInfo latestRegion = latestMap.get(regionId); + + if (latestRegion != null) { + if (currentRegion == null || hasBoundaryChanged(currentRegion, latestRegion)) { + mutations.add(MetaTableAccessor.makePutFromRegionInfo(latestRegion)); + newlyAddedRegions.add(latestRegion); + } + } else { + mutations.add(MetaTableAccessor.makeDeleteFromRegionInfo(currentRegion, + EnvironmentEdgeManager.currentTime())); + } + } + + if (!currentMap.isEmpty() || !latestMap.isEmpty()) { + Set currentTables = + currentMap.values().stream().map(RegionInfo::getTable).collect(Collectors.toSet()); + Set latestTables = + latestMap.values().stream().map(RegionInfo::getTable).collect(Collectors.toSet()); + + Set tablesToDeleteState = new HashSet<>(currentTables); + tablesToDeleteState.removeAll(latestTables); + if (!tablesToDeleteState.isEmpty()) { + LOG.warn( + "The following tables have no regions on storage and WILL BE REMOVED from the meta: {}", + tablesToDeleteState); + this.deletedTables.addAll(tablesToDeleteState); + } + + Set tablesToRestoreState = new HashSet<>(latestTables); + tablesToRestoreState.removeAll(currentTables); + if (!tablesToRestoreState.isEmpty()) { + LOG.info("Adding missing table:state entry for recovered tables: {}", tablesToRestoreState); + for (TableName tableName : tablesToRestoreState) { + TableState tableState = new TableState(tableName, TableState.State.ENABLED); + mutations.add(MetaTableAccessor.makePutFromTableState(tableState, + EnvironmentEdgeManager.currentTime())); + } + } + } + return mutations; + } + + private void applyMutations(Connection connection, List mutations) throws IOException { + List> chunks = Lists.partition(mutations, MUTATION_BATCH_SIZE); + + for (int i = 0; i < chunks.size(); i++) { + List chunk = chunks.get(i); + + List puts = + chunk.stream().filter(m -> m instanceof Put).map(m -> (Put) m).collect(Collectors.toList()); + + List deletes = chunk.stream().filter(m -> m instanceof Delete).map(m -> (Delete) m) + .collect(Collectors.toList()); + + if (!puts.isEmpty()) { + MetaTableAccessor.putsToMetaTable(connection, puts); + } + if (!deletes.isEmpty()) { + MetaTableAccessor.deleteFromMetaTable(connection, deletes); + } + LOG.debug("Successfully processed batch {}/{}", i + 1, chunks.size()); + } + } + + boolean hasBoundaryChanged(RegionInfo region1, RegionInfo region2) { + return !Arrays.equals(region1.getStartKey(), region2.getStartKey()) + || !Arrays.equals(region1.getEndKey(), region2.getEndKey()); + } + + /** + * Scans the backing storage for all regions and returns a list of RegionInfo objects. This method + * scans the filesystem for region directories and reads their .regioninfo files. + * @param connection The HBase connection to use. + * @return List of RegionInfo objects found in the backing storage. + * @throws IOException If there is an error accessing the filesystem or reading region info files. + */ + List scanBackingStorage(Connection connection) throws IOException { + List regions = new ArrayList<>(); + Configuration conf = connection.getConfiguration(); + FileSystem fs = FileSystem.get(conf); + Path rootDir = CommonFSUtils.getRootDir(conf); + Path dataDir = new Path(rootDir, HConstants.BASE_NAMESPACE_DIR); + + LOG.info("Scanning backing storage under: {}", dataDir); + + if (!fs.exists(dataDir)) { + LOG.warn("Data directory does not exist: {}", dataDir); + return regions; + } + + FileStatus[] namespaceDirs = + fs.listStatus(dataDir, path -> !path.getName().matches(HIDDEN_DIR_PATTERN)); + LOG.debug("Found {} namespace directories in data dir", Arrays.stream(namespaceDirs).toList()); + + for (FileStatus nsDir : namespaceDirs) { + String namespaceName = nsDir.getPath().getName(); + if (NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR.equals(namespaceName)) { + LOG.info("Skipping system namespace {}", namespaceName); + continue; + } + try { + List namespaceRegions = scanTablesInNamespace(fs, nsDir.getPath()); + regions.addAll(namespaceRegions); + LOG.debug("Found {} regions in namespace {}", namespaceRegions.size(), + nsDir.getPath().getName()); + } catch (IOException e) { + LOG.error("Failed to scan namespace directory: {}", nsDir.getPath(), e); + } + } + LOG.info("Scanned backing storage and found {} regions", regions.size()); + return regions; + } + + private List scanTablesInNamespace(FileSystem fs, Path namespacePath) + throws IOException { + LOG.debug("Scanning namespace {}", namespacePath.getName()); + List tableDirs = FSUtils.getLocalTableDirs(fs, namespacePath); + + return tableDirs.parallelStream().flatMap(tableDir -> { + try { + List tableRegions = scanRegionsInTable(fs, FSUtils.getRegionDirs(fs, tableDir)); + LOG.debug("Found {} regions in table {} in namespace {}", tableRegions.size(), + tableDir.getName(), namespacePath.getName()); + return tableRegions.stream(); + } catch (IOException e) { + LOG.warn("Failed to scan table directory: {} for namespace {}", tableDir, + namespacePath.getName(), e); + return Stream.empty(); + } + }).toList(); + } + + private List scanRegionsInTable(FileSystem fs, List regionDirs) + throws IOException { + return regionDirs.stream().map(regionDir -> { + String encodedRegionName = regionDir.getName(); + try { + Path regionInfoPath = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE); + if (fs.exists(regionInfoPath)) { + RegionInfo ri = readRegionInfo(fs, regionInfoPath); + if (ri != null && isValidRegionInfo(ri, encodedRegionName)) { + LOG.debug("Found region: {} -> {}", encodedRegionName, ri.getRegionNameAsString()); + return ri; + } else { + LOG.warn("Invalid RegionInfo in file: {}", regionInfoPath); + } + } else { + LOG.debug("No .regioninfo file found in region directory: {}", regionDir); + } + } catch (Exception e) { + LOG.warn("Failed to read region info from directory: {}", encodedRegionName, e); + } + return null; + }).filter(Objects::nonNull).collect(Collectors.toList()); + } + + private boolean isValidRegionInfo(RegionInfo regionInfo, String expectedEncodedName) { + if (!expectedEncodedName.equals(regionInfo.getEncodedName())) { + LOG.warn("RegionInfo encoded name mismatch: directory={}, regioninfo={}", expectedEncodedName, + regionInfo.getEncodedName()); + return false; + } + return true; + } + + private RegionInfo readRegionInfo(FileSystem fs, Path regionInfoPath) { + try (FSDataInputStream inputStream = fs.open(regionInfoPath); + DataInputStream dataInputStream = new DataInputStream(inputStream)) { + return RegionInfo.parseFrom(dataInputStream); + } catch (Exception e) { + LOG.warn("Failed to parse .regioninfo file: {}", regionInfoPath, e); + return null; + } + } + + /** + * Retrieves the current regions from the hbase:meta table. + * @param connection The HBase connection to use. + * @return List of RegionInfo objects representing the current regions in meta. + * @throws IOException If there is an error accessing the meta table. + */ + List getCurrentRegions(Connection connection) throws IOException { + LOG.info("Getting all regions from meta table"); + return MetaTableAccessor.getAllRegions(connection, true); + } + + @Override + protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) { + setState( + org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState.RUNNABLE); + env.getProcedureScheduler().addFront(this); + return false; + } + + @Override + protected void rollbackState(MasterProcedureEnv env, RefreshMetaState refreshMetaState) + throws IOException, InterruptedException { + // No specific rollback needed as it is generally safe to re-run the procedure. + LOG.trace("Rollback not implemented for RefreshMetaProcedure state: {}", refreshMetaState); + } + + @Override + protected RefreshMetaState getState(int stateId) { + return RefreshMetaState.forNumber(stateId); + } + + @Override + protected int getStateId(RefreshMetaState refreshMetaState) { + return refreshMetaState.getNumber(); + } + + @Override + protected RefreshMetaState getInitialState() { + return RefreshMetaState.REFRESH_META_INIT; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + // For now, we'll use a simple approach since we do not need to store any state data + RefreshMetaStateData.Builder builder = RefreshMetaStateData.newBuilder(); + serializer.serialize(builder.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + // For now, we'll use a simple approach since we do not need to store any state data + serializer.deserialize(RefreshMetaStateData.class); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java index 13f458299b90..5b7ab67df0bf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ReadOnlyController.java @@ -109,6 +109,9 @@ public void prePut(ObserverContext c, Pu @Override public void preDelete(ObserverContext c, Delete delete, WALEdit edit) throws IOException { + if (c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + return; + } internalReadOnlyGuard(); } @@ -166,7 +169,9 @@ public boolean preCheckAndPutAfterRowLock( public boolean preCheckAndDelete(ObserverContext c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException { - internalReadOnlyGuard(); + if (!c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + internalReadOnlyGuard(); + } return RegionObserver.super.preCheckAndDelete(c, row, family, qualifier, op, comparator, delete, result); } @@ -174,7 +179,9 @@ public boolean preCheckAndDelete(ObserverContext c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException { - internalReadOnlyGuard(); + if (!c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + internalReadOnlyGuard(); + } return RegionObserver.super.preCheckAndDelete(c, row, filter, delete, result); } @@ -183,7 +190,9 @@ public boolean preCheckAndDeleteAfterRowLock( ObserverContext c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException { - internalReadOnlyGuard(); + if (!c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + internalReadOnlyGuard(); + } return RegionObserver.super.preCheckAndDeleteAfterRowLock(c, row, family, qualifier, op, comparator, delete, result); } @@ -192,7 +201,9 @@ public boolean preCheckAndDeleteAfterRowLock( public boolean preCheckAndDeleteAfterRowLock( ObserverContext c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException { - internalReadOnlyGuard(); + if (!c.getEnvironment().getRegionInfo().getTable().isSystemTable()) { + internalReadOnlyGuard(); + } return RegionObserver.super.preCheckAndDeleteAfterRowLock(c, row, filter, delete, result); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 75bf721ef41e..b32fad50f0f1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -706,4 +706,14 @@ public static boolean createTableDescriptorForTableDirectory(FileSystem fs, Path } return writeTableDescriptor(fs, htd, tableDir, opt.map(Pair::getFirst).orElse(null)) != null; } + + /** + * Invalidates the table descriptor cache. + */ + @Override + public void invalidateTableDescriptorCache() { + LOG.info("Invalidating table descriptor cache."); + this.fsvisited = false; + this.cache.clear(); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedure.java new file mode 100644 index 000000000000..e419d1df6ad7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedure.java @@ -0,0 +1,121 @@ +/* + * 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.master.procedure; + +import static org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.assertProcNotFailed; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; +import org.apache.hadoop.hbase.testclassification.MasterTests; +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; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestRefreshMetaProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefreshMetaProcedure.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private ProcedureExecutor procExecutor; + List activeRegions; + TableName tableName = TableName.valueOf("testRefreshMeta"); + + @Before + public void setup() throws Exception { + TEST_UTIL.getConfiguration().set("USE_META_REPLICAS", "false"); + TEST_UTIL.startMiniCluster(); + procExecutor = TEST_UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor(); + byte[][] splitKeys = + new byte[][] { Bytes.toBytes("split1"), Bytes.toBytes("split2"), Bytes.toBytes("split3") }; + TEST_UTIL.createTable(tableName, Bytes.toBytes("cf"), splitKeys); + TEST_UTIL.waitTableAvailable(tableName); + TEST_UTIL.getAdmin().flush(tableName); + activeRegions = TEST_UTIL.getAdmin().getRegions(tableName); + assertFalse(activeRegions.isEmpty()); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testRefreshMetaProcedureExecutesSuccessfully() { + RefreshMetaProcedure procedure = new RefreshMetaProcedure(procExecutor.getEnvironment()); + long procId = procExecutor.submitProcedure(procedure); + ProcedureTestingUtility.waitProcedure(procExecutor, procId); + assertProcNotFailed(procExecutor.getResult(procId)); + } + + @Test + public void testGetCurrentRegions() throws Exception { + RefreshMetaProcedure procedure = new RefreshMetaProcedure(procExecutor.getEnvironment()); + List regions = procedure.getCurrentRegions(TEST_UTIL.getConnection()); + assertFalse("Should have found regions in meta", regions.isEmpty()); + assertTrue("Should include test table region", + regions.stream().anyMatch(r -> r.getTable().getNameAsString().equals("testRefreshMeta"))); + } + + @Test + public void testScanBackingStorage() throws Exception { + RefreshMetaProcedure procedure = new RefreshMetaProcedure(procExecutor.getEnvironment()); + + List fsRegions = procedure.scanBackingStorage(TEST_UTIL.getConnection()); + + assertTrue("All regions from meta should be found in the storage", + activeRegions.stream().allMatch(reg -> fsRegions.stream() + .anyMatch(r -> r.getRegionNameAsString().equals(reg.getRegionNameAsString())))); + } + + @Test + public void testHasBoundaryChanged() throws Exception { + RefreshMetaProcedure procedure = new RefreshMetaProcedure(procExecutor.getEnvironment()); + RegionInfo region1 = RegionInfoBuilder.newBuilder(tableName) + .setStartKey(Bytes.toBytes("start1")).setEndKey(Bytes.toBytes("end1")).build(); + + RegionInfo region2 = RegionInfoBuilder.newBuilder(tableName) + .setStartKey(Bytes.toBytes("start2")).setEndKey(Bytes.toBytes("end1")).build(); + + RegionInfo region3 = RegionInfoBuilder.newBuilder(tableName) + .setStartKey(Bytes.toBytes("start1")).setEndKey(Bytes.toBytes("end2")).build(); + + assertTrue("Different start keys should have been detected", + procedure.hasBoundaryChanged(region1, region2)); + + assertTrue("Different end keys should have been detected", + procedure.hasBoundaryChanged(region1, region3)); + + assertFalse("Identical boundaries should not have been identified", + procedure.hasBoundaryChanged(region1, region1)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedureIntegration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedureIntegration.java new file mode 100644 index 000000000000..917c12c6513b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshMetaProcedureIntegration.java @@ -0,0 +1,285 @@ +/* + * 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.master.procedure; + +import static org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.assertProcNotFailed; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Stream; +import org.apache.hadoop.fs.FSDataOutputStream; +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.HConstants; +import org.apache.hadoop.hbase.MetaTableAccessor; +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.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.security.access.ReadOnlyController; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MasterTests; +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; + +@Category({ MasterTests.class, LargeTests.class }) +public class TestRefreshMetaProcedureIntegration { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefreshMetaProcedureIntegration.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private Admin admin; + private ProcedureExecutor procExecutor; + private HMaster master; + private HRegionServer regionServer; + + @Before + public void setup() throws Exception { + // Configure the cluster with ReadOnlyController + TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, + ReadOnlyController.class.getName()); + TEST_UTIL.getConfiguration().set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, + ReadOnlyController.class.getName()); + TEST_UTIL.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + ReadOnlyController.class.getName()); + + // Start in active mode + TEST_UTIL.getConfiguration().setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, false); + + TEST_UTIL.startMiniCluster(); + admin = TEST_UTIL.getAdmin(); + procExecutor = TEST_UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor(); + master = TEST_UTIL.getHBaseCluster().getMaster(); + regionServer = TEST_UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer(); + } + + @After + public void tearDown() throws Exception { + if (admin != null) { + admin.close(); + } + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testRestoreMissingRegionInMeta() throws Exception { + + TableName tableName = TableName.valueOf("replicaTestTable"); + + createTableWithData(tableName); + + List activeRegions = admin.getRegions(tableName); + assertTrue("Should have at least 2 regions after split", activeRegions.size() >= 2); + + Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); + RegionInfo regionToRemove = activeRegions.get(0); + admin.unassign(regionToRemove.getRegionName(), false); + Thread.sleep(1000); + + org.apache.hadoop.hbase.client.Delete delete = + new org.apache.hadoop.hbase.client.Delete(regionToRemove.getRegionName()); + metaTable.delete(delete); + metaTable.close(); + + List regionsAfterDrift = admin.getRegions(tableName); + assertEquals("Should have one less region in meta after simulating drift", + activeRegions.size() - 1, regionsAfterDrift.size()); + + setReadOnlyMode(true); + + boolean writeBlocked = false; + try { + Table readOnlyTable = TEST_UTIL.getConnection().getTable(tableName); + Put testPut = new Put(Bytes.toBytes("test_readonly")); + testPut.addColumn(Bytes.toBytes("cf1"), Bytes.toBytes("qual"), Bytes.toBytes("should_fail")); + readOnlyTable.put(testPut); + readOnlyTable.close(); + } catch (Exception e) { + if (e.getMessage().contains("Operation not allowed in Read-Only Mode")) { + writeBlocked = true; + } + } + assertTrue("Write operations should be blocked in read-only mode", writeBlocked); + + Long procId = admin.refreshMeta(); + + waitForProcedureCompletion(procId); + + List regionsAfterRefresh = admin.getRegions(tableName); + assertEquals("Missing regions should be restored by refresh_meta", activeRegions.size(), + regionsAfterRefresh.size()); + + boolean regionRestored = regionsAfterRefresh.stream() + .anyMatch(r -> r.getRegionNameAsString().equals(regionToRemove.getRegionNameAsString())); + assertTrue("Missing region should be restored by refresh_meta", regionRestored); + + setReadOnlyMode(false); + + Table activeTable = TEST_UTIL.getConnection().getTable(tableName); + Put testPut = new Put(Bytes.toBytes("test_active_again")); + testPut.addColumn(Bytes.toBytes("cf1"), Bytes.toBytes("qual"), + Bytes.toBytes("active_mode_again")); + activeTable.put(testPut); + activeTable.close(); + } + + @Test + public void testPhantomTableCleanup() throws Exception { + TableName table1 = TableName.valueOf("table1"); + TableName phantomTable = TableName.valueOf("phantomTable"); + createTableWithData(table1); + createTableWithData(phantomTable); + + assertTrue("Table1 should have multiple regions", admin.getRegions(table1).size() >= 2); + assertTrue("phantomTable should have multiple regions", + admin.getRegions(phantomTable).size() >= 2); + + deleteTableFromFilesystem(phantomTable); + List tablesBeforeRefresh = Arrays.asList(admin.listTableNames()); + assertTrue("phantomTable should still be listed before refresh_meta", + tablesBeforeRefresh.contains(phantomTable)); + assertTrue("Table1 should still be listed", tablesBeforeRefresh.contains(table1)); + + setReadOnlyMode(true); + Long procId = admin.refreshMeta(); + waitForProcedureCompletion(procId); + + List tablesAfterRefresh = Arrays.asList(admin.listTableNames()); + + assertFalse("phantomTable should be removed after refresh_meta", + tablesAfterRefresh.contains(phantomTable)); + assertTrue("Table1 should still be listed", tablesAfterRefresh.contains(table1)); + assertTrue("phantomTable should have no regions after refresh_meta", + admin.getRegions(phantomTable).isEmpty()); + setReadOnlyMode(false); + } + + @Test + public void testRestoreTableStateForOrphanRegions() throws Exception { + TableName tableName = TableName.valueOf("t1"); + createTableInFilesystem(tableName); + + assertEquals("No tables should exist", 0, + Stream.of(admin.listTableNames()).filter(tn -> tn.equals(tableName)).count()); + + setReadOnlyMode(true); + Long procId = admin.refreshMeta(); + waitForProcedureCompletion(procId); + + TableState tableState = MetaTableAccessor.getTableState(admin.getConnection(), tableName); + assert tableState != null; + assertEquals("Table state should be ENABLED", TableState.State.ENABLED, tableState.getState()); + assertEquals("The list should show the new table from the FS", 1, + Stream.of(admin.listTableNames()).filter(tn -> tn.equals(tableName)).count()); + assertFalse("Should have at least 1 region", admin.getRegions(tableName).isEmpty()); + setReadOnlyMode(false); + } + + private void createTableInFilesystem(TableName tableName) throws IOException { + FileSystem fs = TEST_UTIL.getTestFileSystem(); + Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()); + Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName); + fs.mkdirs(tableDir); + + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); + TEST_UTIL.getHBaseCluster().getMaster().getTableDescriptors() + .update(builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf1")).build(), false); + + Path regionDir = new Path(tableDir, "dab6d1e1c88787c13b97647f11b2c907"); + Path regionInfoFile = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE); + fs.mkdirs(regionDir); + + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).setStartKey(new byte[0]) + .setEndKey(new byte[0]).setRegionId(1757100253228L).build(); + byte[] regionInfoContent = RegionInfo.toDelimitedByteArray(regionInfo); + try (FSDataOutputStream out = fs.create(regionInfoFile, true)) { + out.write(regionInfoContent); + } + } + + private void deleteTableFromFilesystem(TableName tableName) throws IOException { + FileSystem fs = TEST_UTIL.getTestFileSystem(); + Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()); + Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName); + if (fs.exists(tableDir)) { + fs.delete(tableDir, true); + } + } + + private void createTableWithData(TableName tableName) throws Exception { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); + builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf1")); + byte[] splitKeyBytes = Bytes.toBytes("split_key"); + admin.createTable(builder.build(), new byte[][] { splitKeyBytes }); + TEST_UTIL.waitTableAvailable(tableName); + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + for (int i = 0; i < 100; i++) { + Put put = new Put(Bytes.toBytes("row_" + String.format("%05d", i))); + put.addColumn(Bytes.toBytes("cf1"), Bytes.toBytes("qual"), Bytes.toBytes("value_" + i)); + table.put(put); + } + } + admin.flush(tableName); + } + + private void waitForProcedureCompletion(Long procId) { + assertTrue("Procedure ID should be positive", procId > 0); + TEST_UTIL.waitFor(1000, () -> { + try { + return procExecutor.isFinished(procId); + } catch (Exception e) { + return false; + } + }); + assertProcNotFailed(procExecutor.getResult(procId)); + } + + private void setReadOnlyMode(boolean isReadOnly) { + TEST_UTIL.getConfiguration().setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, + isReadOnly); + notifyConfigurationObservers(); + } + + private void notifyConfigurationObservers() { + master.getConfigurationManager().notifyAllObservers(TEST_UTIL.getConfiguration()); + regionServer.getConfigurationManager().notifyAllObservers(TEST_UTIL.getConfiguration()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java index b0e3eb051fc6..1ac785e7276a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java @@ -1011,4 +1011,8 @@ public boolean isReplicationPeerModificationEnabled() throws IOException { return admin.isReplicationPeerModificationEnabled(); } + @Override + public Long refreshMeta() throws IOException { + return admin.refreshMeta(); + } } diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 2b3b1086f75e..389d3a65fd08 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -1955,6 +1955,12 @@ def flush_master_store() def list_tables_by_state(isEnabled) @admin.listTableNamesByState(isEnabled).map(&:getNameAsString) end + + #---------------------------------------------------------------------------------------------- + # Refresh hbase:meta table by syncing with the backing storage + def refresh_meta() + @admin.refreshMeta() + end end # rubocop:enable Metrics/ClassLength end diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index 5774ff3073b9..e8de631932fc 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -493,6 +493,7 @@ def self.exception_handler(hide_traceback) decommission_regionservers recommission_regionserver truncate_region + refresh_meta ], # TODO: remove older hlog_roll command aliases: { diff --git a/hbase-shell/src/main/ruby/shell/commands/refresh_meta.rb b/hbase-shell/src/main/ruby/shell/commands/refresh_meta.rb new file mode 100644 index 000000000000..8c5acb49dc54 --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/refresh_meta.rb @@ -0,0 +1,43 @@ +# +# 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. +# + +module Shell + module Commands + class RefreshMeta < Command + def help + <<-EOF +Refresh the hbase:meta table by syncing with backing storage. +This command is used in Read Replica clusters to pick up new +tables and regions from the shared storage. +Examples: + + hbase> refresh_meta + +The command returns a procedure ID that can be used to track the progress +of the meta table refresh operation. +EOF + end + + def command + proc_id = admin.refresh_meta + formatter.row(["Refresh meta procedure submitted. Procedure ID: #{proc_id}"]) + proc_id + end + end + end +end diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java index 76a8b41481be..cfd53156cc4f 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java @@ -1374,6 +1374,11 @@ public void restoreBackupSystemTable(String snapshotName) throws IOException { throw new NotImplementedException("restoreBackupSystemTable not supported in ThriftAdmin"); } + @Override + public Long refreshMeta() throws IOException { + throw new NotImplementedException("refreshMeta not supported in ThriftAdmin"); + } + @Override public boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures) throws IOException { From 56691db3141bcf3f9116fdba21b381355e069662 Mon Sep 17 00:00:00 2001 From: Anuj Sharma Date: Tue, 16 Sep 2025 00:49:05 +0530 Subject: [PATCH 06/35] HBASE-29328: Implement new HBase command: refresh_hfiles (#7149) * CDPD-84463 Add ruby shell commands for refresh_hfiles * [CDPD-84466] Add hbase-client API code to refresh_hfiles * CDPD-84465 Add protobuf messages for refresh_hfiles * Add refreshHfile function in master rpc service and make call to its function * CDPD-82553 Add function in Region Server to refresh Hfiles * Add nonceGroup and nonce for the Master RPC request * Refactor code with proper name for function * Add region Server Procedure and callables * Remove the refreshHFiles function which was intended to call as RS RPC As we will be calling it through procedure framework * Remove the unwanted comments * Add line mistakenly removed in admin.proto * Correct the wrong comment in Event Types * Apply Spotless * Address the review comments having small code changes * Add separate function for master service caller * Add retry mechanism for refresh_hfiles, send exception if retry threshold get breached Also handle scenario in case the region is not online * Add tablename into RefreshHFilesTableProcedureStateData * CDPD-88507, CDPD-88508 Add procdure suspport for namespace as parameter and no parameter * nit: Add meaningful name to method and remove comments * Return exception if user is updating system table or reserved namespaces * Send exception if tablename or namespace is invalid Also remove redundant TODOs * Add gatekeeper method to prevent execution because command before master initialize * Return exception in case both TABLE_NAME and NAMESPACE is provided in arguments * Run Spotless * Add unit tests for refreshHfiles Procedure and admin calls * Make the newly added HFiles available for reading immediately * Revert "Make the newly added HFiles available for reading immediately" This reverts commit c25cc9a070ab240a406756413d33236e98a7a630. * Address review comments * Create test base class to avoid code duplication * Add integration test which enable readonly mode before refresh * Added test rule and rebased the upstream * Apply spotless --- .../org/apache/hadoop/hbase/client/Admin.java | 23 ++ .../hbase/client/AdminOverAsyncAdmin.java | 15 ++ .../hadoop/hbase/client/AsyncAdmin.java | 15 ++ .../hadoop/hbase/client/AsyncHBaseAdmin.java | 15 ++ .../hbase/client/RawAsyncHBaseAdmin.java | 50 +++++ .../main/protobuf/server/master/Master.proto | 14 ++ .../server/master/MasterProcedure.proto | 19 ++ .../hadoop/hbase/executor/EventType.java | 8 +- .../hadoop/hbase/executor/ExecutorType.java | 3 +- .../apache/hadoop/hbase/master/HMaster.java | 74 +++++++ .../hbase/master/MasterRpcServices.java | 20 ++ .../RefreshHFilesRegionProcedure.java | 208 ++++++++++++++++++ .../RefreshHFilesTableProcedure.java | 165 ++++++++++++++ .../procedure/TableProcedureInterface.java | 3 +- .../hbase/master/procedure/TableQueue.java | 1 + .../hbase/regionserver/HRegionServer.java | 4 + .../regionserver/RefreshHFilesCallable.java | 69 ++++++ .../hadoop/hbase/TestRefreshHFilesBase.java | 161 ++++++++++++++ .../client/TestRefreshHFilesFromClient.java | 139 ++++++++++++ .../procedure/TestRefreshHFilesProcedure.java | 116 ++++++++++ ...efreshHFilesProcedureWithReadOnlyConf.java | 124 +++++++++++ .../hbase/rsgroup/VerifyingRSGroupAdmin.java | 15 ++ hbase-shell/src/main/ruby/hbase/admin.rb | 20 ++ hbase-shell/src/main/ruby/shell.rb | 1 + .../ruby/shell/commands/refresh_hfiles.rb | 64 ++++++ .../hbase/thrift2/client/ThriftAdmin.java | 15 ++ 26 files changed, 1358 insertions(+), 3 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesRegionProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesTableProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RefreshHFilesCallable.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/TestRefreshHFilesBase.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRefreshHFilesFromClient.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedure.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedureWithReadOnlyConf.java create mode 100644 hbase-shell/src/main/ruby/shell/commands/refresh_hfiles.rb diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 8622e1c88774..7370983d01a8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -2708,6 +2708,29 @@ List getLogEntries(Set serverNames, String logType, Server */ Long refreshMeta() throws IOException; + /** + * Refresh HFiles for the table + * @param tableName table to refresh HFiles for + * @return ID of the procedure started for refreshing HFiles + * @throws IOException if a remote or network exception occurs + */ + Long refreshHFiles(final TableName tableName) throws IOException; + + /** + * Refresh HFiles for all the tables under given namespace + * @param namespace Namespace for which we should call refresh HFiles for all tables under it + * @return ID of the procedure started for refreshing HFiles + * @throws IOException if a remote or network exception occurs + */ + Long refreshHFiles(final String namespace) throws IOException; + + /** + * Refresh HFiles for all the tables + * @return ID of the procedure started for refreshing HFiles + * @throws IOException if a remote or network exception occurs + */ + Long refreshHFiles() throws IOException; + @InterfaceAudience.Private void restoreBackupSystemTable(String snapshotName) throws IOException; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java index 37d46feb41ad..ca0ffb67329f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java @@ -1158,6 +1158,21 @@ public Long refreshMeta() throws IOException { return get(admin.refreshMeta()); } + @Override + public Long refreshHFiles(final TableName tableName) throws IOException { + return get(admin.refreshHFiles(tableName)); + } + + @Override + public Long refreshHFiles(final String namespace) throws IOException { + return get(admin.refreshHFiles(namespace)); + } + + @Override + public Long refreshHFiles() throws IOException { + return get(admin.refreshHFiles()); + } + @Override public void restoreBackupSystemTable(String snapshotName) throws IOException { get(admin.restoreBackupSystemTable(snapshotName)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index e1d3aadfed34..114d103ce03c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -1895,6 +1895,21 @@ CompletableFuture> getLogEntries(Set serverNames, Str */ CompletableFuture refreshMeta(); + /** + * Refresh HFiles for the table + */ + CompletableFuture refreshHFiles(final TableName tableName); + + /** + * Refresh HFiles for all the tables under given namespace + */ + CompletableFuture refreshHFiles(final String namespace); + + /** + * Refresh HFiles for all the tables + */ + CompletableFuture refreshHFiles(); + @InterfaceAudience.Private CompletableFuture restoreBackupSystemTable(String snapshotName); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 26abe68402ca..4cf02d91f0e9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -1026,6 +1026,21 @@ public CompletableFuture refreshMeta() { return wrap(rawAdmin.refreshMeta()); } + @Override + public CompletableFuture refreshHFiles(final TableName tableName) { + return wrap(rawAdmin.refreshHFiles(tableName)); + } + + @Override + public CompletableFuture refreshHFiles(final String namespace) { + return wrap(rawAdmin.refreshHFiles(namespace)); + } + + @Override + public CompletableFuture refreshHFiles() { + return wrap(rawAdmin.refreshHFiles()); + } + @Override public CompletableFuture restoreBackupSystemTable(String snapshotName) { return wrap(rawAdmin.restoreBackupSystemTable(snapshotName)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 9454d2826533..58409251cef3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -263,6 +263,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RefreshHFilesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RefreshHFilesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RefreshMetaRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RefreshMetaResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ReopenTableRegionsRequest; @@ -4700,6 +4702,14 @@ MasterProtos.RestoreBackupSystemTableResponse> procedureCall(request, new RestoreBackupSystemTableProcedureBiConsumer()); } + private CompletableFuture internalRefershHFiles(RefreshHFilesRequest request) { + return this. newMasterCaller() + .action((controller, stub) -> this. call( + controller, stub, request, MasterService.Interface::refreshHFiles, + RefreshHFilesResponse::getProcId)) + .call(); + } + @Override public CompletableFuture refreshMeta() { RefreshMetaRequest.Builder request = RefreshMetaRequest.newBuilder(); @@ -4710,4 +4720,44 @@ public CompletableFuture refreshMeta() { RefreshMetaResponse::getProcId)) .call(); } + + @Override + public CompletableFuture refreshHFiles(final TableName tableName) { + if (tableName.isSystemTable()) { + LOG.warn("Refreshing HFiles for system table {} is not allowed", tableName.getNameAsString()); + throw new IllegalArgumentException( + "Not allowed to refresh HFiles for system table '" + tableName.getNameAsString() + "'"); + } + // Request builder + RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder(); + request.setTableName(ProtobufUtil.toProtoTableName(tableName)); + request.setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()); + return internalRefershHFiles(request.build()); + } + + @Override + public CompletableFuture refreshHFiles(final String namespace) { + if ( + namespace.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) + || namespace.equals(NamespaceDescriptor.BACKUP_NAMESPACE_NAME_STR) + ) { + LOG.warn("Refreshing HFiles for reserve namespace {} is not allowed", namespace); + throw new IllegalArgumentException( + "Not allowed to refresh HFiles for reserve namespace '" + namespace + "'"); + } + // Request builder + RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder(); + request.setNamespace(namespace); + request.setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()); + return internalRefershHFiles(request.build()); + } + + @Override + public CompletableFuture refreshHFiles() { + // Request builder + RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder(); + // Set nonce + request.setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()); + return internalRefershHFiles(request.build()); + } } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto index 40f6a1518f13..c774a93605ab 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto @@ -807,6 +807,17 @@ message ModifyColumnStoreFileTrackerResponse { optional uint64 proc_id = 1; } +message RefreshHFilesRequest { + optional TableName table_name = 1; + optional string namespace = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; +} + +message RefreshHFilesResponse { + optional uint64 proc_id = 1; +} + message FlushMasterStoreRequest {} message FlushMasterStoreResponse {} @@ -1314,6 +1325,9 @@ service MasterService { rpc RefreshMeta(RefreshMetaRequest) returns(RefreshMetaResponse); + + rpc RefreshHFiles(RefreshHFilesRequest) + returns(RefreshHFilesResponse); } // HBCK Service definitions. diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto index aa79ff474c35..56086aed29e3 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto @@ -876,3 +876,22 @@ enum RefreshMetaState { message RefreshMetaStateData { } + +enum RefreshHFilesTableProcedureState { + REFRESH_HFILES_PREPARE = 1; + REFRESH_HFILES_REFRESH_REGION = 2; + REFRESH_HFILES_FINISH = 3; +} + +message RefreshHFilesTableProcedureStateData { + optional TableName table_name = 1; + optional string namespace_name = 2; +} + +message RefreshHFilesRegionProcedureStateData { + required RegionInfo region = 1; +} + +message RefreshHFilesRegionParameter { + required RegionInfo region = 1; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java index fee132b7a4d3..90985923b72f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java @@ -309,7 +309,13 @@ public enum EventType { * RS log roll.
* RS_LOG_ROLL */ - RS_LOG_ROLL(91, ExecutorType.RS_LOG_ROLL); + RS_LOG_ROLL(91, ExecutorType.RS_LOG_ROLL), + + /** + * RS refresh hfiles for a region.
+ * RS_REFRESH_HFILES + */ + RS_REFRESH_HFILES(92, ExecutorType.RS_REFRESH_HFILES); private final int code; private final ExecutorType executor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java index 668cd701c0d9..e2d357fbee61 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java @@ -57,7 +57,8 @@ public enum ExecutorType { RS_SNAPSHOT_OPERATIONS(36), RS_FLUSH_OPERATIONS(37), RS_RELOAD_QUOTAS_OPERATIONS(38), - RS_LOG_ROLL(39); + RS_LOG_ROLL(39), + RS_REFRESH_HFILES(39); ExecutorType(int value) { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index caca0fcef73f..a6b64bd1021f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.PleaseRestartMasterException; import org.apache.hadoop.hbase.RegionMetrics; @@ -170,6 +171,7 @@ import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait; import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher; +import org.apache.hadoop.hbase.master.procedure.RefreshHFilesTableProcedure; import org.apache.hadoop.hbase.master.procedure.RefreshMetaProcedure; import org.apache.hadoop.hbase.master.procedure.ReloadQuotasProcedure; import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure; @@ -4668,4 +4670,76 @@ protected String getDescription() { } }); } + + public Long refreshHfiles(final TableName tableName, final long nonceGroup, final long nonce) + throws IOException { + checkInitialized(); + + if (!tableDescriptors.exists(tableName)) { + LOG.info("RefreshHfilesProcedure failed because table {} does not exist", + tableName.getNameAsString()); + throw new TableNotFoundException(tableName); + } + + return MasterProcedureUtil + .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + @Override + protected void run() throws IOException { + LOG.info("Submitting RefreshHfilesTableProcedure for table {}", + tableName.getNameAsString()); + submitProcedure( + new RefreshHFilesTableProcedure(procedureExecutor.getEnvironment(), tableName)); + } + + @Override + protected String getDescription() { + return "RefreshHfilesProcedure for a table"; + } + }); + } + + public Long refreshHfiles(final String namespace, final long nonceGroup, final long nonce) + throws IOException { + checkInitialized(); + + try { + this.clusterSchemaService.getNamespace(namespace); + } catch (IOException e) { + LOG.info("RefreshHfilesProcedure failed because namespace {} does not exist", namespace); + throw new NamespaceNotFoundException(namespace); + } + + return MasterProcedureUtil + .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + @Override + protected void run() throws IOException { + LOG.info("Submitting RefreshHfilesProcedure for namespace {}", namespace); + submitProcedure( + new RefreshHFilesTableProcedure(procedureExecutor.getEnvironment(), namespace)); + } + + @Override + protected String getDescription() { + return "RefreshHfilesProcedure for namespace"; + } + }); + } + + public Long refreshHfiles(final long nonceGroup, final long nonce) throws IOException { + checkInitialized(); + + return MasterProcedureUtil + .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + @Override + protected void run() throws IOException { + LOG.info("Submitting RefreshHfilesProcedure for all tables"); + submitProcedure(new RefreshHFilesTableProcedure(procedureExecutor.getEnvironment())); + } + + @Override + protected String getDescription() { + return "RefreshHfilesProcedure for all tables"; + } + }); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 472a8f863982..a020378d7e27 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -3753,4 +3753,24 @@ public MasterProtos.RefreshMetaResponse refreshMeta(RpcController controller, throw new ServiceException(ioe); } } + + @Override + public MasterProtos.RefreshHFilesResponse refreshHFiles(RpcController controller, + MasterProtos.RefreshHFilesRequest request) throws ServiceException { + try { + Long procId; + if (request.hasTableName()) { + procId = server.refreshHfiles(ProtobufUtil.toTableName(request.getTableName()), + request.getNonceGroup(), request.getNonce()); + } else if (request.hasNamespace()) { + procId = + server.refreshHfiles(request.getNamespace(), request.getNonceGroup(), request.getNonce()); + } else { + procId = server.refreshHfiles(request.getNonceGroup(), request.getNonce()); + } + return MasterProtos.RefreshHFilesResponse.newBuilder().setProcId(procId).build(); + } catch (IOException ioe) { + throw new ServiceException(ioe); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesRegionProcedure.java new file mode 100644 index 000000000000..0680826a4c29 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesRegionProcedure.java @@ -0,0 +1,208 @@ +/* + * 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.master.procedure; + +import java.io.IOException; +import java.util.Optional; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.master.assignment.RegionStateNode; +import org.apache.hadoop.hbase.master.assignment.RegionStates; +import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException; +import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.hadoop.hbase.procedure2.ProcedureEvent; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureUtil; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; +import org.apache.hadoop.hbase.regionserver.RefreshHFilesCallable; +import org.apache.hadoop.hbase.util.RetryCounter; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; + +/** + * A master-side procedure that handles refreshing HFiles (store files) for a specific region in + * HBase. It performs remote procedure dispatch to the RegionServer hosting the region and manages + * retries, suspensions, and timeouts as needed. This procedure ensures safe execution by verifying + * the region state, handling remote operation results, and applying retry mechanisms in case of + * failures. It gives the call to {@link RefreshHFilesCallable} which gets executed on region + * server. + */ + +@InterfaceAudience.Private +public class RefreshHFilesRegionProcedure extends Procedure + implements TableProcedureInterface, + RemoteProcedureDispatcher.RemoteProcedure { + private static final Logger LOG = LoggerFactory.getLogger(RefreshHFilesRegionProcedure.class); + private RegionInfo region; + private ProcedureEvent event; + private boolean dispatched; + private boolean succ; + private RetryCounter retryCounter; + + public RefreshHFilesRegionProcedure() { + } + + public RefreshHFilesRegionProcedure(RegionInfo region) { + this.region = region; + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + MasterProcedureProtos.RefreshHFilesRegionProcedureStateData data = + serializer.deserialize(MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.class); + this.region = ProtobufUtil.toRegionInfo(data.getRegion()); + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.Builder builder = + MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.newBuilder(); + builder.setRegion(ProtobufUtil.toRegionInfo(region)); + serializer.serialize(builder.build()); + } + + @Override + protected boolean abort(MasterProcedureEnv env) { + return false; + } + + @Override + protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException { + throw new UnsupportedOperationException(); + } + + private void setTimeoutForSuspend(MasterProcedureEnv env, String reason) { + if (retryCounter == null) { + retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration()); + } + long backoff = retryCounter.getBackoffTimeAndIncrementAttempts(); + LOG.warn("{} can not run currently because {}, wait {} ms to retry", this, reason, backoff); + setTimeout(Math.toIntExact(backoff)); + setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT); + skipPersistence(); + } + + @Override + protected Procedure[] execute(MasterProcedureEnv env) + throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException { + if (dispatched) { + if (succ) { + return null; + } + dispatched = false; + } + + RegionStates regionStates = env.getAssignmentManager().getRegionStates(); + RegionStateNode regionNode = regionStates.getRegionStateNode(region); + + if (regionNode.getProcedure() != null) { + setTimeoutForSuspend(env, String.format("region %s has a TRSP attached %s", + region.getRegionNameAsString(), regionNode.getProcedure())); + throw new ProcedureSuspendedException(); + } + + if (!regionNode.isInState(RegionState.State.OPEN)) { + LOG.warn("State of region {} is not OPEN. Skip {} ...", region, this); + setTimeoutForSuspend(env, String.format("region state of %s is %s", + region.getRegionNameAsString(), regionNode.getState())); + throw new ProcedureSuspendedException(); + } + + ServerName targetServer = regionNode.getRegionLocation(); + if (targetServer == null) { + setTimeoutForSuspend(env, + String.format("target server of region %s is null", region.getRegionNameAsString())); + throw new ProcedureSuspendedException(); + } + + try { + env.getRemoteDispatcher().addOperationToNode(targetServer, this); + dispatched = true; + event = new ProcedureEvent<>(this); + event.suspendIfNotReady(this); + throw new ProcedureSuspendedException(); + } catch (FailedRemoteDispatchException e) { + setTimeoutForSuspend(env, "Failed send request to " + targetServer); + throw new ProcedureSuspendedException(); + } + } + + @Override + public TableOperationType getTableOperationType() { + return TableOperationType.REFRESH_HFILES; + } + + @Override + public TableName getTableName() { + return region.getTable(); + } + + @Override + public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) { + complete(env, error); + } + + @Override + public void remoteOperationCompleted(MasterProcedureEnv env) { + complete(env, null); + } + + @Override + public void remoteCallFailed(MasterProcedureEnv env, ServerName serverName, IOException e) { + complete(env, e); + } + + private void complete(MasterProcedureEnv env, Throwable error) { + if (isFinished()) { + LOG.info("This procedure {} is already finished. Skip the rest of the processes", + this.getProcId()); + return; + } + if (event == null) { + LOG.warn("procedure event for {} is null, maybe the procedure is created when recovery", + getProcId()); + return; + } + if (error == null) { + succ = true; + } + event.wake(env.getProcedureScheduler()); + event = null; + } + + @Override + public Optional remoteCallBuild(MasterProcedureEnv env, + ServerName serverName) { + MasterProcedureProtos.RefreshHFilesRegionParameter.Builder builder = + MasterProcedureProtos.RefreshHFilesRegionParameter.newBuilder(); + builder.setRegion(ProtobufUtil.toRegionInfo(region)); + return Optional + .of(new RSProcedureDispatcher.ServerOperation(this, getProcId(), RefreshHFilesCallable.class, + builder.build().toByteArray(), env.getMasterServices().getMasterActiveTime())); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesTableProcedure.java new file mode 100644 index 000000000000..cb225d23b0d9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesTableProcedure.java @@ -0,0 +1,165 @@ +/* + * 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.master.procedure; + +import java.io.IOException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshHFilesTableProcedureState; + +@InterfaceAudience.Private +public class RefreshHFilesTableProcedure + extends AbstractStateMachineTableProcedure { + private static final Logger LOG = LoggerFactory.getLogger(RefreshHFilesTableProcedure.class); + + private TableName tableName; + private String namespaceName; + + public RefreshHFilesTableProcedure() { + super(); + } + + public RefreshHFilesTableProcedure(MasterProcedureEnv env) { + super(env); + } + + public RefreshHFilesTableProcedure(MasterProcedureEnv env, TableName tableName) { + super(env); + this.tableName = tableName; + } + + public RefreshHFilesTableProcedure(MasterProcedureEnv env, String namespaceName) { + super(env); + this.namespaceName = namespaceName; + } + + @Override + public TableOperationType getTableOperationType() { + return TableOperationType.REFRESH_HFILES; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + MasterProcedureProtos.RefreshHFilesTableProcedureStateData.Builder builder = + MasterProcedureProtos.RefreshHFilesTableProcedureStateData.newBuilder(); + if (tableName != null && namespaceName == null) { + builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); + } else if (tableName == null && namespaceName != null) { + builder.setNamespaceName(namespaceName); + } + serializer.serialize(builder.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + MasterProcedureProtos.RefreshHFilesTableProcedureStateData data = + serializer.deserialize(MasterProcedureProtos.RefreshHFilesTableProcedureStateData.class); + if (data.hasTableName() && !data.hasNamespaceName()) { + this.tableName = ProtobufUtil.toTableName(data.getTableName()); + } else if (!data.hasTableName() && data.hasNamespaceName()) { + this.namespaceName = data.getNamespaceName(); + } + } + + @Override + public TableName getTableName() { + if (tableName != null && namespaceName == null) { + return tableName; + } + return DUMMY_NAMESPACE_TABLE_NAME; + } + + @Override + protected RefreshHFilesTableProcedureState getInitialState() { + return RefreshHFilesTableProcedureState.REFRESH_HFILES_PREPARE; + } + + @Override + protected int getStateId(RefreshHFilesTableProcedureState state) { + return state.getNumber(); + } + + @Override + protected RefreshHFilesTableProcedureState getState(int stateId) { + return RefreshHFilesTableProcedureState.forNumber(stateId); + } + + @Override + protected void rollbackState(MasterProcedureEnv env, RefreshHFilesTableProcedureState state) + throws IOException, InterruptedException { + // Refresh HFiles is idempotent operation hence rollback is not needed + LOG.trace("Rollback not implemented for RefreshHFilesTableProcedure state: {}", state); + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, RefreshHFilesTableProcedureState state) { + LOG.info("Executing RefreshHFilesTableProcedureState state: {}", state); + + try { + return switch (state) { + case REFRESH_HFILES_PREPARE -> prepare(env); + case REFRESH_HFILES_REFRESH_REGION -> refreshHFiles(env); + case REFRESH_HFILES_FINISH -> finish(); + default -> throw new UnsupportedOperationException("Unhandled state: " + state); + }; + } catch (Exception ex) { + LOG.error("Error in RefreshHFilesTableProcedure state {}", state, ex); + setFailure("RefreshHFilesTableProcedure", ex); + return Flow.NO_MORE_STATE; + } + } + + private Flow prepare(final MasterProcedureEnv env) { + setNextState(RefreshHFilesTableProcedureState.REFRESH_HFILES_REFRESH_REGION); + return Flow.HAS_MORE_STATE; + } + + private void refreshHFilesForTable(final MasterProcedureEnv env, TableName tableName) { + addChildProcedure(env.getAssignmentManager().getTableRegions(tableName, true).stream() + .map(RefreshHFilesRegionProcedure::new).toArray(RefreshHFilesRegionProcedure[]::new)); + } + + private Flow refreshHFiles(final MasterProcedureEnv env) throws IOException { + if (tableName != null && namespaceName == null) { + refreshHFilesForTable(env, tableName); + } else if (tableName == null && namespaceName != null) { + env.getMasterServices().listTableNamesByNamespace(namespaceName) + .forEach(table -> refreshHFilesForTable(env, table)); + } else { + env.getMasterServices().getTableDescriptors().getAll().values().stream() + .map(TableDescriptor::getTableName).filter(table -> !table.isSystemTable()) + .forEach(table -> refreshHFilesForTable(env, table)); + } + + setNextState(RefreshHFilesTableProcedureState.REFRESH_HFILES_FINISH); + return Flow.HAS_MORE_STATE; + } + + private Flow finish() { + return Flow.NO_MORE_STATE; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java index c5c7ec602eab..7014b7cf01f3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java @@ -51,7 +51,8 @@ public enum TableOperationType { REGION_GC, MERGED_REGIONS_GC/* region operations */, REGION_TRUNCATE, - RESTORE_BACKUP_SYSTEM_TABLE + RESTORE_BACKUP_SYSTEM_TABLE, + REFRESH_HFILES, } /** Returns the name of the table the procedure is operating on */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableQueue.java index 7be4c4b1810e..a58db4d5ed69 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableQueue.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableQueue.java @@ -73,6 +73,7 @@ static boolean requireTableExclusiveLock(TableProcedureInterface proc) { case MERGED_REGIONS_GC: case REGION_SNAPSHOT: case REGION_TRUNCATE: + case REFRESH_HFILES: return false; default: break; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 6e6717d4238a..00ec12e89927 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1972,6 +1972,10 @@ executorService.new ExecutorConfig().setExecutorType(ExecutorType.RS_RELOAD_QUOT final int logRollThreads = conf.getInt("hbase.regionserver.executor.log.roll.threads", 1); executorService.startExecutorService(executorService.new ExecutorConfig() .setExecutorType(ExecutorType.RS_LOG_ROLL).setCorePoolSize(logRollThreads)); + final int rsRefreshHFilesThreads = + conf.getInt("hbase.regionserver.executor.refresh.hfiles.threads", 3); + executorService.startExecutorService(executorService.new ExecutorConfig() + .setExecutorType(ExecutorType.RS_REFRESH_HFILES).setCorePoolSize(rsRefreshHFilesThreads)); Threads.setDaemonThreadRunning(this.walRoller, getName() + ".logRoller", uncaughtExceptionHandler); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RefreshHFilesCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RefreshHFilesCallable.java new file mode 100644 index 000000000000..f864bf4c2c0d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RefreshHFilesCallable.java @@ -0,0 +1,69 @@ +/* + * 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 java.io.IOException; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.executor.EventType; +import org.apache.hadoop.hbase.procedure2.BaseRSProcedureCallable; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; + +/** + * This is a RegionServer-side callable used by the HBase Master Procedure framework to perform the + * actual HFiles refresh operation on a specific region. It is dispatched from the + * {@link org.apache.hadoop.hbase.master.procedure.RefreshHFilesRegionProcedure} to the RegionServer + * and executes the logic to refresh store files in each store of the region. + */ + +@InterfaceAudience.Private +public class RefreshHFilesCallable extends BaseRSProcedureCallable { + private static final Logger LOG = LoggerFactory.getLogger(RefreshHFilesCallable.class); + + private RegionInfo regionInfo; + + @Override + protected void doCall() throws Exception { + HRegion region = rs.getRegion(regionInfo.getEncodedName()); + LOG.debug("Starting refreshHfiles operation on region {}", region); + + try { + for (Store store : region.getStores()) { + store.refreshStoreFiles(); + } + } catch (IOException ioe) { + LOG.warn("Exception while trying to refresh store files: ", ioe); + } + } + + @Override + protected void initParameter(byte[] parameter) throws Exception { + MasterProcedureProtos.RefreshHFilesRegionParameter param = + MasterProcedureProtos.RefreshHFilesRegionParameter.parseFrom(parameter); + this.regionInfo = ProtobufUtil.toRegionInfo(param.getRegion()); + } + + @Override + public EventType getEventType() { + return EventType.RS_REFRESH_HFILES; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRefreshHFilesBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRefreshHFilesBase.java new file mode 100644 index 000000000000..3d9f188242de --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRefreshHFilesBase.java @@ -0,0 +1,161 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_RETRIES_NUMBER; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.RefreshHFilesTableProcedure; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.security.access.ReadOnlyController; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestRefreshHFilesBase { + + private static final Logger LOG = LoggerFactory.getLogger(TestRefreshHFilesBase.class); + + protected static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + protected Admin admin; + protected HMaster master; + List regionServers; + protected SingleProcessHBaseCluster cluster; + protected ProcedureExecutor procExecutor; + protected static Configuration conf; + protected static final TableName TEST_TABLE = TableName.valueOf("testRefreshHFilesTable"); + protected static final String TEST_NAMESPACE = "testRefreshHFilesNamespace"; + protected static final byte[] TEST_FAMILY = Bytes.toBytes("testRefreshHFilesCF1"); + + protected void createTableAndWait(TableName table, byte[] cf) + throws IOException, InterruptedException { + TEST_UTIL.createTable(table, cf); + TEST_UTIL.waitTableAvailable(table); + } + + protected void createTableInNamespaceAndWait(String namespace, TableName table, byte[] cf) + throws IOException, InterruptedException { + TableName fqTableName = TableName.valueOf(namespace + table.getNameAsString()); + TEST_UTIL.createTable(fqTableName, cf); + TEST_UTIL.waitTableAvailable(fqTableName); + } + + protected void deleteTable(TableName table) throws IOException { + TEST_UTIL.deleteTableIfAny(table); + } + + protected void createNamespace(String namespace) throws RuntimeException { + try { + final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespace).build(); + // Create the namespace if it doesn’t exist + if ( + Arrays.stream(admin.listNamespaceDescriptors()) + .noneMatch(ns -> ns.getName().equals(namespace)) + ) { + admin.createNamespace(nsd); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected void deleteNamespace(String namespace) { + try { + // List table in namespace + TableName[] tables = admin.listTableNamesByNamespace(namespace); + for (TableName t : tables) { + TEST_UTIL.deleteTableIfAny(t); + } + // Now delete the namespace + admin.deleteNamespace(namespace); + } catch (Exception e) { + LOG.debug( + "Unable to delete namespace " + namespace + " post test execution. This isn't a failure"); + } + } + + protected void submitProcedureAndAssertNotFailed(RefreshHFilesTableProcedure procedure) { + long procId = procExecutor.submitProcedure(procedure); + ProcedureTestingUtility.waitProcedure(procExecutor, procId); + ProcedureTestingUtility.assertProcNotFailed(procExecutor.getResult(procId)); + } + + protected void setReadOnlyMode(boolean isReadOnly) { + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, isReadOnly); + notifyConfigurationObservers(); + } + + private void notifyConfigurationObservers() { + master.getConfigurationManager().notifyAllObservers(TEST_UTIL.getConfiguration()); + for (HRegionServer rs : regionServers) { + rs.getConfigurationManager().notifyAllObservers(TEST_UTIL.getConfiguration()); + } + } + + private void setupReadOnlyConf(boolean addReadOnlyConf) { + if (!addReadOnlyConf) return; + conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ReadOnlyController.class.getName()); + conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ReadOnlyController.class.getName()); + // Keep ReadOnly property to false at the beginning so that create table succeed. + conf.setBoolean(HConstants.HBASE_GLOBAL_READONLY_ENABLED_KEY, false); + } + + protected void baseSetup(boolean addReadOnlyConf) throws Exception { + conf = TEST_UTIL.getConfiguration(); + // Shorten the run time of failed unit tests by limiting retries and the session timeout + // threshold + conf.setInt(HBASE_CLIENT_RETRIES_NUMBER, 1); + conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); + conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 60000); + conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000); + + setupReadOnlyConf(addReadOnlyConf); + + try { + // Start the test cluster + cluster = TEST_UTIL.startMiniCluster(1); + admin = TEST_UTIL.getAdmin(); + procExecutor = TEST_UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor(); + master = TEST_UTIL.getHBaseCluster().getMaster(); + regionServers = cluster.getRegionServerThreads().stream() + .map(JVMClusterUtil.RegionServerThread::getRegionServer).collect(Collectors.toList()); + } catch (Exception e) { + TEST_UTIL.shutdownMiniCluster(); + throw new RuntimeException(e); + } + } + + protected void baseTearDown() throws Exception { + if (admin != null) { + admin.close(); + } + TEST_UTIL.shutdownMiniCluster(); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRefreshHFilesFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRefreshHFilesFromClient.java new file mode 100644 index 000000000000..c134c6c9804f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRefreshHFilesFromClient.java @@ -0,0 +1,139 @@ +/* + * 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.client; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.NamespaceNotFoundException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.TestRefreshHFilesBase; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MediumTests.class, ClientTests.class }) +public class TestRefreshHFilesFromClient extends TestRefreshHFilesBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefreshHFilesFromClient.class); + + private static final TableName TEST_NONEXISTENT_TABLE = + TableName.valueOf("testRefreshHFilesNonExistentTable"); + private static final String TEST_NONEXISTENT_NAMESPACE = "testRefreshHFilesNonExistentNamespace"; + + @Before + public void setup() throws Exception { + baseSetup(false); + } + + @After + public void tearDown() throws Exception { + baseTearDown(); + } + + @Test + public void testRefreshHFilesForTable() throws Exception { + try { + // Create table in default namespace + createTableAndWait(TEST_TABLE, TEST_FAMILY); + + // RefreshHFiles for table + Long procId = admin.refreshHFiles(TEST_TABLE); + assertTrue(procId >= 0); + } catch (Exception e) { + Assert.fail("RefreshHFilesForTable Should Not Throw Exception: " + e); + throw new RuntimeException(e); + } finally { + // Delete table name post test execution + deleteTable(TEST_TABLE); + } + } + + // Not creating table hence refresh should throw exception + @Test(expected = TableNotFoundException.class) + public void testRefreshHFilesForNonExistentTable() throws Exception { + // RefreshHFiles for table + admin.refreshHFiles(TEST_NONEXISTENT_TABLE); + } + + @Test + public void testRefreshHFilesForNamespace() throws Exception { + try { + createNamespace(TEST_NAMESPACE); + + // Create table under test namespace + createTableInNamespaceAndWait(TEST_NAMESPACE, TEST_TABLE, TEST_FAMILY); + + // RefreshHFiles for namespace + Long procId = admin.refreshHFiles(TEST_NAMESPACE); + assertTrue(procId >= 0); + + } catch (Exception e) { + Assert.fail("RefreshHFilesForAllNamespace Should Not Throw Exception: " + e); + throw new RuntimeException(e); + } finally { + // Delete namespace post test execution + // This will delete all tables under namespace hence no explicit table + // deletion for table under namespace is needed. + deleteNamespace(TEST_NAMESPACE); + } + } + + @Test(expected = NamespaceNotFoundException.class) + public void testRefreshHFilesForNonExistentNamespace() throws Exception { + // RefreshHFiles for namespace + admin.refreshHFiles(TEST_NONEXISTENT_NAMESPACE); + } + + @Test + public void testRefreshHFilesForAllTables() throws Exception { + try { + // Create table in default namespace + createTableAndWait(TEST_TABLE, TEST_FAMILY); + + // Create test namespace + createNamespace(TEST_NAMESPACE); + + // Create table under test namespace + createTableInNamespaceAndWait(TEST_NAMESPACE, TEST_TABLE, TEST_FAMILY); + + // RefreshHFiles for all the tables + Long procId = admin.refreshHFiles(); + assertTrue(procId >= 0); + + } catch (Exception e) { + Assert.fail("RefreshHFilesForAllTables Should Not Throw Exception: " + e); + throw new RuntimeException(e); + } finally { + // Delete table name post test execution + deleteTable(TEST_TABLE); + + // Delete namespace post test execution + // This will delete all tables under namespace hence no explicit table + // deletion for table under namespace is needed. + deleteNamespace(TEST_NAMESPACE); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedure.java new file mode 100644 index 000000000000..9bb7bf181c3b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedure.java @@ -0,0 +1,116 @@ +/* + * 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.master.procedure; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TestRefreshHFilesBase; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestRefreshHFilesProcedure extends TestRefreshHFilesBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefreshHFilesProcedure.class); + + @Before + public void setup() throws Exception { + baseSetup(false); + } + + @After + public void tearDown() throws Exception { + baseTearDown(); + } + + @Test + public void testRefreshHFilesProcedureForTable() throws IOException { + try { + // Create table in default namespace + createTableAndWait(TEST_TABLE, TEST_FAMILY); + + RefreshHFilesTableProcedure procedure = + new RefreshHFilesTableProcedure(procExecutor.getEnvironment(), TEST_TABLE); + submitProcedureAndAssertNotFailed(procedure); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + // Delete table name post test execution + deleteTable(TEST_TABLE); + } + } + + @Test + public void testRefreshHFilesProcedureForNamespace() { + try { + createNamespace(TEST_NAMESPACE); + + // Create table under test namespace + createTableInNamespaceAndWait(TEST_NAMESPACE, TEST_TABLE, TEST_FAMILY); + + RefreshHFilesTableProcedure procedure = + new RefreshHFilesTableProcedure(procExecutor.getEnvironment(), TEST_NAMESPACE); + submitProcedureAndAssertNotFailed(procedure); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + // Delete namespace post test execution + // This will delete all tables under namespace hence no explicit table + // deletion for table under namespace is needed. + deleteNamespace(TEST_NAMESPACE); + } + } + + @Test + public void testRefreshHFilesProcedureForAllTables() throws IOException { + try { + // Create table in default namespace + createTableAndWait(TEST_TABLE, TEST_FAMILY); + + // Create test namespace + createNamespace(TEST_NAMESPACE); + + // Create table under test namespace + createTableInNamespaceAndWait(TEST_NAMESPACE, TEST_TABLE, TEST_FAMILY); + + RefreshHFilesTableProcedure procedure = + new RefreshHFilesTableProcedure(procExecutor.getEnvironment()); + submitProcedureAndAssertNotFailed(procedure); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + // Delete table name post test execution + deleteTable(TEST_TABLE); + + // Delete namespace post test execution + // This will delete all tables under namespace hence no explicit table + // deletion for table under namespace is needed. + deleteNamespace(TEST_NAMESPACE); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedureWithReadOnlyConf.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedureWithReadOnlyConf.java new file mode 100644 index 000000000000..351081d04fc4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRefreshHFilesProcedureWithReadOnlyConf.java @@ -0,0 +1,124 @@ +/* + * 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.master.procedure; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TestRefreshHFilesBase; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestRefreshHFilesProcedureWithReadOnlyConf extends TestRefreshHFilesBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRefreshHFilesProcedureWithReadOnlyConf.class); + + @Before + public void setup() throws Exception { + // When true is passed only setup for readonly property is done. + // The initial ReadOnly property will be false for table creation + baseSetup(true); + } + + @After + public void tearDown() throws Exception { + baseTearDown(); + } + + @Test + public void testRefreshHFilesProcedureForTable() throws IOException { + try { + // Create table in default namespace + createTableAndWait(TEST_TABLE, TEST_FAMILY); + + setReadOnlyMode(true); + RefreshHFilesTableProcedure procedure = + new RefreshHFilesTableProcedure(procExecutor.getEnvironment(), TEST_TABLE); + submitProcedureAndAssertNotFailed(procedure); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + setReadOnlyMode(false); + // Delete table name post test execution + deleteTable(TEST_TABLE); + } + } + + @Test + public void testRefreshHFilesProcedureForNamespace() { + try { + createNamespace(TEST_NAMESPACE); + + // Create table under test namespace + createTableInNamespaceAndWait(TEST_NAMESPACE, TEST_TABLE, TEST_FAMILY); + + setReadOnlyMode(true); + RefreshHFilesTableProcedure procedure = + new RefreshHFilesTableProcedure(procExecutor.getEnvironment(), TEST_NAMESPACE); + submitProcedureAndAssertNotFailed(procedure); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + setReadOnlyMode(false); + // Delete namespace post test execution + // This will delete all tables under namespace hence no explicit table + // deletion for table under namespace is needed. + deleteNamespace(TEST_NAMESPACE); + } + } + + @Test + public void testRefreshHFilesProcedureForAllTables() throws IOException { + try { + // Create table in default namespace + createTableAndWait(TEST_TABLE, TEST_FAMILY); + + // Create test namespace + createNamespace(TEST_NAMESPACE); + + // Create table under test namespace + createTableInNamespaceAndWait(TEST_NAMESPACE, TEST_TABLE, TEST_FAMILY); + + setReadOnlyMode(true); + RefreshHFilesTableProcedure procedure = + new RefreshHFilesTableProcedure(procExecutor.getEnvironment()); + submitProcedureAndAssertNotFailed(procedure); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + setReadOnlyMode(false); + // Delete table name post test execution + deleteTable(TEST_TABLE); + + // Delete namespace post test execution + // This will delete all tables under namespace hence no explicit table + // deletion for table under namespace is needed. + deleteNamespace(TEST_NAMESPACE); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java index 1ac785e7276a..cf4f1f0e838c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java @@ -1015,4 +1015,19 @@ public boolean isReplicationPeerModificationEnabled() throws IOException { public Long refreshMeta() throws IOException { return admin.refreshMeta(); } + + @Override + public Long refreshHFiles(final TableName tableName) throws IOException { + return admin.refreshHFiles(tableName); + } + + @Override + public Long refreshHFiles(final String namespace) throws IOException { + return admin.refreshHFiles(namespace); + } + + @Override + public Long refreshHFiles() throws IOException { + return admin.refreshHFiles(); + } } diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 389d3a65fd08..2e79dc8fdcd8 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -1961,6 +1961,26 @@ def list_tables_by_state(isEnabled) def refresh_meta() @admin.refreshMeta() end + + #---------------------------------------------------------------------------------------------- + # Refresh HFiles for the table + def refresh_hfiles(args = {}) + table_name = args.fetch(TABLE_NAME, nil) + namespace = args.fetch(NAMESPACE, nil) + if namespace && table_name + raise ArgumentError, "Specify either a TABLE_NAME or a NAMESPACE, not both" + elsif namespace == "" || table_name == "" + raise ArgumentError, "TABLE_NAME or NAMESPACE cannot be empty string" + elsif namespace.is_a?(Array) || table_name.is_a?(Array) + raise ArgumentError, "TABLE_NAME or NAMESPACE must be a single string, not an array" + elsif namespace + @admin.refreshHFiles(namespace) + elsif table_name + @admin.refreshHFiles(org.apache.hadoop.hbase.TableName.valueOf(table_name)) + else + @admin.refreshHFiles() + end + end end # rubocop:enable Metrics/ClassLength end diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index e8de631932fc..03984272ea36 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -494,6 +494,7 @@ def self.exception_handler(hide_traceback) recommission_regionserver truncate_region refresh_meta + refresh_hfiles ], # TODO: remove older hlog_roll command aliases: { diff --git a/hbase-shell/src/main/ruby/shell/commands/refresh_hfiles.rb b/hbase-shell/src/main/ruby/shell/commands/refresh_hfiles.rb new file mode 100644 index 000000000000..1ec0c74381aa --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/refresh_hfiles.rb @@ -0,0 +1,64 @@ +# +# 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. +# + +module Shell + module Commands + class RefreshHfiles < Command + def help + return <<-EOF +Refresh HFiles/storefiles for table(s) or namespaces. + +Allowed Syntax: +hbase> refresh_hfiles +hbase> refresh_hfiles 'TABLE_NAME' => 'test_table' +hbase> refresh_hfiles 'TABLE_NAME' => 'namespace:test_table' +hbase> refresh_hfiles 'NAMESPACE' => 'test_namespace' + +Behavior: +- Without any argument, it refreshes HFiles for all user tables in HBase. +- If 'TABLE_NAME' is provided: Refreshes HFiles for that specific table. + - If provided without a namespace qualifier (e.g., 'TABLE_NAME' => 'test_table'), + it refreshes HFiles for that table in the default namespace. + - If provided with a namespace qualifier (e.g., 'TABLE_NAME' => 'namespace:test_table'), + it refreshes HFiles for the table in the specified namespace. +- With 'NAMESPACE', it refreshes HFiles for all tables in the given namespace. +On successful submission, it returns the procedure ID (procId). Otherwise, it throws an exception. + +Important Note: +This command should ideally be run on a read-replica cluster, + and only after successfully executing refresh_meta. + +Not Allowed: +hbase> refresh_hfiles 'TABLE_NAME' => 'test_table', 'NAMESPACE' => 'test_namespace' + +Passing both 'TABLE_NAME' and 'NAMESPACE' is not allowed to avoid ambiguity. +Otherwise, it is unclear whether the user intends to: +1. Refresh HFiles for 'test_table' under 'test_namespace', +or +2. Refresh HFiles for 'test_table' under the default namespace and + all tables under 'test_namespace'. +To prevent such confusion, only one argument should be provided per command. + +EOF + end + def command(args = {}) + admin.refresh_hfiles(args) + end + end + end +end diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java index cfd53156cc4f..f29d6b5771d5 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java @@ -1391,4 +1391,19 @@ public boolean isReplicationPeerModificationEnabled() throws IOException { throw new NotImplementedException( "isReplicationPeerModificationEnabled not supported in ThriftAdmin"); } + + @Override + public Long refreshHFiles(final TableName tableName) throws IOException { + throw new NotImplementedException("refreshHFiles not supported in ThriftAdmin"); + } + + @Override + public Long refreshHFiles(final String namespace) throws IOException { + throw new NotImplementedException("refreshHFiles not supported in ThriftAdmin"); + } + + @Override + public Long refreshHFiles() throws IOException { + throw new NotImplementedException("refreshHFiles not supported in ThriftAdmin"); + } } From 8bdbb62ed4e02a4b26f55771106ebacb0da52eb8 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Fri, 19 Sep 2025 11:39:59 -0500 Subject: [PATCH 07/35] HBASE-29579: AssignmentManager is trying to pick up the other cluster's meta table after HbckChore run (#7304) --- .../apache/hadoop/hbase/master/HMaster.java | 7 +++ .../hadoop/hbase/master/hbck/HbckChore.java | 3 +- .../hadoop/hbase/util/FSTableDescriptors.java | 3 +- .../org/apache/hadoop/hbase/util/FSUtils.java | 19 ++++++++ .../hbase/master/TestMasterMetrics.java | 33 ++++++++++++++ .../master/assignment/TestHbckChore.java | 43 +++++++++++++++++++ .../hbase/util/TestFSTableDescriptors.java | 33 ++++++++++++++ 7 files changed, 139 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index a6b64bd1021f..87595e6bb1f9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -3140,6 +3140,13 @@ public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet