Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Connection;
Expand Down Expand Up @@ -169,7 +170,7 @@ private List<String> getLogFilesForNewBackup(Map<String, Long> olderTimestamps,
LOG.debug("currentLogFile: " + log.getPath().toString());
if (AbstractFSWALProvider.isMetaFile(log.getPath())) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
LOG.debug("Skip {} log file: {}", MetaTableName.getInstance(), log.getPath().getName());
}
continue;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.ServerName;
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.Put;
Expand Down Expand Up @@ -131,7 +131,7 @@ public static void updateMetaWithFavoredNodesInfo(
puts.add(put);
}
}
try (Table table = connection.getTable(TableName.META_TABLE_NAME)) {
try (Table table = connection.getTable(MetaTableName.getInstance())) {
table.put(puts);
}
LOG.info("Added " + puts.size() + " region favored nodes in META");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hadoop.hbase.CatalogFamilyFormat;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -170,9 +171,10 @@ private void processMetaRecord(Result result) throws IOException {
* Initialize the region assignment snapshot by scanning the hbase:meta table
*/
public void initialize() throws IOException {
LOG.info("Start to scan the hbase:meta for the current region assignment " + "snappshot");
LOG.info("Start to scan {} for the current region assignment snapshot",
MetaTableName.getInstance());
// Scan hbase:meta to pick up user regions
try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME);
try (Table metaTable = connection.getTable(MetaTableName.getInstance());
ResultScanner scanner = metaTable.getScanner(HConstants.CATALOG_FAMILY)) {
for (;;) {
Result result = scanner.next();
Expand All @@ -187,7 +189,8 @@ public void initialize() throws IOException {
}
}
}
LOG.info("Finished to scan the hbase:meta for the current region assignment" + "snapshot");
LOG.info("Finished scanning {} for the current region assignment snapshot",
MetaTableName.getInstance());
}

private void addRegion(RegionInfo regionInfo) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
Expand Down Expand Up @@ -68,7 +69,7 @@ public static void setup() {
// Create regions
List<RegionInfo> allRegions = new ArrayList<>();
for (int i = 0; i < NUM_REGIONS; i++) {
TableName tableName = i < 3 ? TableName.META_TABLE_NAME : NON_META_TABLE_NAME;
TableName tableName = i < 3 ? MetaTableName.getInstance() : NON_META_TABLE_NAME;
byte[] startKey = new byte[1];
startKey[0] = (byte) i;
byte[] endKey = new byte[1];
Expand All @@ -95,7 +96,7 @@ public void testMetaTableIsolation() {
}

private boolean isMetaTableIsolated(BalancerClusterState cluster) {
return isTableIsolated(cluster, TableName.META_TABLE_NAME, "Meta");
return isTableIsolated(cluster, MetaTableName.getInstance(), "Meta");
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
Expand Down Expand Up @@ -72,7 +73,7 @@ public static void setup() {
for (int i = 0; i < NUM_REGIONS; i++) {
TableName tableName;
if (i < 1) {
tableName = TableName.META_TABLE_NAME;
tableName = MetaTableName.getInstance();
} else if (i < 10) {
tableName = SYSTEM_TABLE_NAME;
} else {
Expand Down Expand Up @@ -116,7 +117,7 @@ public void testTableIsolationAndReplicaDistribution() {
* Validates whether all meta table regions are isolated.
*/
private boolean isMetaTableIsolated(BalancerClusterState cluster) {
return isTableIsolated(cluster, TableName.META_TABLE_NAME, "Meta");
return isTableIsolated(cluster, MetaTableName.getInstance(), "Meta");
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableReg
final AsyncTable<AdvancedScanResultConsumer> metaTable, final TableName tableName,
final boolean excludeOfflinedSplitParents) {
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
if (TableName.META_TABLE_NAME.equals(tableName)) {
if (MetaTableName.getInstance().equals(tableName)) {
future.completeExceptionally(new IOException(
"This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import static org.apache.hadoop.hbase.HConstants.NINES;
import static org.apache.hadoop.hbase.HConstants.USE_META_REPLICAS;
import static org.apache.hadoop.hbase.HConstants.ZEROES;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
import static org.apache.hadoop.hbase.client.ConnectionConfiguration.HBASE_CLIENT_META_CACHE_INVALIDATE_INTERVAL;
Expand Down Expand Up @@ -52,6 +51,7 @@
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -238,14 +238,15 @@ private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations
CatalogReplicaLoadBalanceSimpleSelector.class.getName());

this.metaReplicaSelector = CatalogReplicaLoadBalanceSelectorFactory
.createSelector(replicaSelectorClass, META_TABLE_NAME, conn, () -> {
.createSelector(replicaSelectorClass, MetaTableName.getInstance(), conn, () -> {
int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS;
try {
RegionLocations metaLocations = conn.registry.getMetaRegionLocations()
.get(conn.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS);
numOfReplicas = metaLocations.size();
} catch (Exception e) {
LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e);
LOG.error("Failed to get table {}'s region replication, ",
MetaTableName.getInstance(), e);
}
return numOfReplicas;
});
Expand Down Expand Up @@ -427,7 +428,7 @@ private void locateInMeta(TableName tableName, LocateRequest req) {
// do nothing
}

conn.getTable(META_TABLE_NAME).scan(scan, new AdvancedScanResultConsumer() {
conn.getTable(MetaTableName.getInstance()).scan(scan, new AdvancedScanResultConsumer() {

private boolean completeNormally = false;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hbase.client;

import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.REGION_NAMES_KEY;
import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.SERVER_NAME_KEY;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
Expand All @@ -36,6 +35,7 @@
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -217,7 +217,7 @@ void clearCache(TableName tableName) {
new TableSpanBuilder(conn).setName("AsyncRegionLocator.clearCache").setTableName(tableName);
TraceUtil.trace(() -> {
LOG.debug("Clear meta cache for {}", tableName);
if (tableName.equals(META_TABLE_NAME)) {
if (tableName.equals(MetaTableName.getInstance())) {
metaRegionLocator.clearCache();
} else {
nonMetaRegionLocator.clearCache(tableName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.hbase.ClientMetaTableAccessor;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;

Expand Down Expand Up @@ -63,7 +64,7 @@ public CompletableFuture<List<HRegionLocation>> getAllRegionLocations() {
.thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
}
CompletableFuture<List<HRegionLocation>> future = ClientMetaTableAccessor
.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), tableName);
.getTableHRegionLocations(conn.getTable(MetaTableName.getInstance()), tableName);
addListener(future, (locs, error) -> locs.forEach(loc -> {
// the cache assumes that all locations have a serverName. only add if that's true
if (loc.getServerName() != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
Expand Down Expand Up @@ -120,7 +121,7 @@ private static int checkReplicaId(int regionId) {
this.replicaId = checkReplicaId(replicaId);
this.offLine = offLine;
this.regionName = RegionInfo.createRegionName(this.tableName, this.startKey, this.regionId,
this.replicaId, !this.tableName.equals(TableName.META_TABLE_NAME));
this.replicaId, !this.tableName.equals(MetaTableName.getInstance()));
this.encodedName = RegionInfo.encodeRegionName(this.regionName);
this.hashCode = generateHashCode(this.tableName, this.startKey, this.endKey, this.regionId,
this.replicaId, this.offLine, this.regionName);
Expand Down Expand Up @@ -232,7 +233,7 @@ public boolean containsRow(byte[] row) {
/** Returns true if this region is a meta region */
@Override
public boolean isMetaRegion() {
return tableName.equals(TableName.META_TABLE_NAME);
return tableName.equals(MetaTableName.getInstance());
}

/** Returns True if has been split and has daughters. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.client;

import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;

Expand Down Expand Up @@ -57,6 +56,7 @@
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionMetrics;
Expand Down Expand Up @@ -403,7 +403,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
AsyncAdminBuilderBase builder) {
this.connection = connection;
this.retryTimer = retryTimer;
this.metaTable = connection.getTable(META_TABLE_NAME);
this.metaTable = connection.getTable(MetaTableName.getInstance());
this.rpcTimeoutNs = builder.rpcTimeoutNs;
this.operationTimeoutNs = builder.operationTimeoutNs;
this.pauseNs = builder.pauseNs;
Expand Down Expand Up @@ -995,7 +995,7 @@ List<RegionInfo>> adminCall(controller, stub,

@Override
public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName) {
if (tableName.equals(META_TABLE_NAME)) {
if (tableName.equals(MetaTableName.getInstance())) {
return connection.registry.getMetaRegionLocations()
.thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion)
.collect(Collectors.toList()));
Expand Down Expand Up @@ -1286,7 +1286,7 @@ private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFa
* List all region locations for the specific table.
*/
private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName) {
if (TableName.META_TABLE_NAME.equals(tableName)) {
if (MetaTableName.getInstance().equals(tableName)) {
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> {
if (err != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.ByteArrayHashKey;
Expand Down Expand Up @@ -431,7 +432,7 @@ static byte[] toByteArray(RegionInfo ri) {
*/
static String prettyPrint(final String encodedRegionName) {
if (encodedRegionName.equals("1028785192")) {
return encodedRegionName + "/hbase:meta";
return encodedRegionName + "/" + MetaTableName.getInstance();
}
return encodedRegionName;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.client;

import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
Expand All @@ -43,7 +44,7 @@ public class RegionInfoBuilder {
// TODO: How come Meta regions still do not have encoded region names? Fix.
// hbase:meta,,1.1588230740 should be the hbase:meta first region name.
public static final RegionInfo FIRST_META_REGIONINFO =
new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
new MutableRegionInfo(1L, MetaTableName.getInstance(), RegionInfo.DEFAULT_REPLICA_ID);

private final TableName tableName;
private byte[] startKey = HConstants.EMPTY_START_ROW;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
Expand Down Expand Up @@ -82,7 +82,7 @@ public static String getRegionNameAsStringForDisplay(RegionInfo ri, Configuratio
*/
public static byte[] getRegionNameForDisplay(RegionInfo ri, Configuration conf) {
boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
if (displayKey || ri.getTable().equals(TableName.META_TABLE_NAME)) {
if (displayKey || ri.getTable().equals(MetaTableName.getInstance())) {
return ri.getRegionName();
} else {
// create a modified regionname with the startkey replaced but preserving
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.HBaseException;
Expand Down Expand Up @@ -616,7 +617,7 @@ private ModifyableTableDescriptor(final TableName name,
families.forEach(c -> this.families.put(c.getName(), ColumnFamilyDescriptorBuilder.copy(c)));
this.values.putAll(values);
this.values.put(IS_META_KEY,
new Bytes(Bytes.toBytes(Boolean.toString(name.equals(TableName.META_TABLE_NAME)))));
new Bytes(Bytes.toBytes(Boolean.toString(name.equals(MetaTableName.getInstance())))));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import java.security.PrivilegedExceptionAction;
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.Connection;
Expand Down Expand Up @@ -73,7 +73,7 @@ private static void injectFault() throws ServiceException {
future.completeExceptionally(ProtobufUtil.handleRemoteException(injectedException));
return future;
}
AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
AsyncTable<?> table = conn.getTable(MetaTableName.getInstance());
table.<AuthenticationProtos.AuthenticationService.Interface,
AuthenticationProtos.GetAuthenticationTokenResponse> coprocessorService(
AuthenticationProtos.AuthenticationService::newStub,
Expand Down Expand Up @@ -102,7 +102,7 @@ static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn) throws
try {
injectFault();

meta = conn.getTable(TableName.META_TABLE_NAME);
meta = conn.getTable(MetaTableName.getInstance());
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
AuthenticationProtos.AuthenticationService.BlockingInterface service =
AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MetaTableName;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ServerTask;
Expand Down Expand Up @@ -3325,7 +3326,7 @@ public static String toLockJson(List<LockServiceProtos.LockedResource> lockedRes
long regionId = proto.getRegionId();
int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : defaultReplicaId;
if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) {
if (tableName.equals(MetaTableName.getInstance()) && replicaId == defaultReplicaId) {
return RegionInfoBuilder.FIRST_META_REGIONINFO;
}
byte[] startKey = null;
Expand Down
Loading