HBASE-25121 Refactor MetaTableAccessor.addRegionsToMeta and its usage places (#2476)

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Duo Zhang 2020-10-05 21:29:55 +08:00 committed by GitHub
parent 23ce91819a
commit 9ba90e1679
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 217 additions and 233 deletions

View File

@ -27,7 +27,7 @@ import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell.Type; import org.apache.hadoop.hbase.Cell.Type;
import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType; import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
@ -247,7 +247,7 @@ public final class MetaTableAccessor {
throws IOException { throws IOException {
RowFilter rowFilter = RowFilter rowFilter =
new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName)); new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
Scan scan = getMetaScan(connection, 1); Scan scan = getMetaScan(connection.getConfiguration(), 1);
scan.setFilter(rowFilter); scan.setFilter(rowFilter);
try (Table table = getMetaHTable(connection); try (Table table = getMetaHTable(connection);
ResultScanner resultScanner = table.getScanner(scan)) { ResultScanner resultScanner = table.getScanner(scan)) {
@ -320,24 +320,23 @@ public final class MetaTableAccessor {
* @param tableName bytes of table's name * @param tableName bytes of table's name
* @return configured Scan object * @return configured Scan object
*/ */
public static Scan getScanForTableName(Connection connection, TableName tableName) { public static Scan getScanForTableName(Configuration conf, TableName tableName) {
// Start key is just the table name with delimiters // Start key is just the table name with delimiters
byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION); byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
// Stop key appends the smallest possible char to the table name // Stop key appends the smallest possible char to the table name
byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION); byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
Scan scan = getMetaScan(connection, -1); Scan scan = getMetaScan(conf, -1);
scan.withStartRow(startKey); scan.withStartRow(startKey);
scan.withStopRow(stopKey); scan.withStopRow(stopKey);
return scan; return scan;
} }
private static Scan getMetaScan(Connection connection, int rowUpperLimit) { private static Scan getMetaScan(Configuration conf, int rowUpperLimit) {
Scan scan = new Scan(); Scan scan = new Scan();
int scannerCaching = connection.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING, int scannerCaching = conf.getInt(HConstants.HBASE_META_SCANNER_CACHING,
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING); HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS, if (conf.getBoolean(HConstants.USE_META_REPLICAS, HConstants.DEFAULT_USE_META_REPLICAS)) {
HConstants.DEFAULT_USE_META_REPLICAS)) {
scan.setConsistency(Consistency.TIMELINE); scan.setConsistency(Consistency.TIMELINE);
} }
if (rowUpperLimit > 0) { if (rowUpperLimit > 0) {
@ -469,7 +468,7 @@ public final class MetaTableAccessor {
@Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows, @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
final ClientMetaTableAccessor.Visitor visitor) throws IOException { final ClientMetaTableAccessor.Visitor visitor) throws IOException {
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE; int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
Scan scan = getMetaScan(connection, rowUpperLimit); Scan scan = getMetaScan(connection.getConfiguration(), rowUpperLimit);
for (byte[] family : type.getFamilies()) { for (byte[] family : type.getFamilies()) {
scan.addFamily(family); scan.addFamily(family);
@ -525,7 +524,7 @@ public final class MetaTableAccessor {
private static RegionInfo getClosestRegionInfo(Connection connection, private static RegionInfo getClosestRegionInfo(Connection connection,
@NonNull final TableName tableName, @NonNull final byte[] row) throws IOException { @NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false); byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
Scan scan = getMetaScan(connection, 1); Scan scan = getMetaScan(connection.getConfiguration(), 1);
scan.setReversed(true); scan.setReversed(true);
scan.withStartRow(searchRow); scan.withStartRow(searchRow);
try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) { try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) {
@ -733,37 +732,6 @@ public final class MetaTableAccessor {
} }
} }
/**
* Deletes some replica columns corresponding to replicas for the passed rows
* @param metaRows rows in hbase:meta
* @param replicaIndexToDeleteFrom the replica ID we would start deleting from
* @param numReplicasToRemove how many replicas to remove
* @param connection connection we're using to access meta table
*/
public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows,
int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
throws IOException {
int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
for (byte[] row : metaRows) {
long now = EnvironmentEdgeManager.currentTime();
Delete deleteReplicaLocations = new Delete(row);
for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getServerColumn(i), now);
deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getSeqNumColumn(i), now);
deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getStartCodeColumn(i), now);
deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getServerNameColumn(i), now);
deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getRegionStateColumn(i), now);
}
deleteFromMetaTable(connection, deleteReplicaLocations);
}
}
public static Put addRegionStateToPut(Put put, RegionState.State state) throws IOException { public static Put addRegionStateToPut(Put put, RegionState.State state) throws IOException {
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow()) put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.STATE_QUALIFIER) .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.STATE_QUALIFIER)
@ -804,22 +772,6 @@ public final class MetaTableAccessor {
} }
} }
/**
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
* does not add its daughter's as different rows, but adds information about the daughters in the
* same row as the parent. Use
* {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)} if
* you want to do that.
* @param connection connection we're using
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
@VisibleForTesting
public static void addRegionToMeta(Connection connection, RegionInfo regionInfo)
throws IOException {
addRegionsToMeta(connection, Collections.singletonList(regionInfo), 1);
}
/** /**
* Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is
* CLOSED. * CLOSED.
@ -845,7 +797,9 @@ public final class MetaTableAccessor {
int regionReplication, long ts) throws IOException { int regionReplication, long ts) throws IOException {
List<Put> puts = new ArrayList<>(); List<Put> puts = new ArrayList<>();
for (RegionInfo regionInfo : regionInfos) { for (RegionInfo regionInfo : regionInfos) {
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) { if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
continue;
}
Put put = makePutFromRegionInfo(regionInfo, ts); Put put = makePutFromRegionInfo(regionInfo, ts);
// New regions are added with initial state of CLOSED. // New regions are added with initial state of CLOSED.
addRegionStateToPut(put, RegionState.State.CLOSED); addRegionStateToPut(put, RegionState.State.CLOSED);
@ -856,7 +810,6 @@ public final class MetaTableAccessor {
} }
puts.add(put); puts.add(put);
} }
}
putsToMetaTable(connection, puts); putsToMetaTable(connection, puts);
LOG.info("Added {} regions to meta.", puts.size()); LOG.info("Added {} regions to meta.", puts.size());
} }

View File

@ -311,8 +311,7 @@ public class CatalogFamilyFormat {
* @param replicaId the replicaId of the region * @param replicaId the replicaId of the region
* @return a byte[] for state qualifier * @return a byte[] for state qualifier
*/ */
@VisibleForTesting public static byte[] getRegionStateColumn(int replicaId) {
static byte[] getRegionStateColumn(int replicaId) {
return replicaId == 0 ? HConstants.STATE_QUALIFIER : return replicaId == 0 ? HConstants.STATE_QUALIFIER :
Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER + Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER +
String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));

View File

@ -48,6 +48,8 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
@ -545,6 +547,83 @@ public class RegionStateStore {
LOG.debug("Overwritten regions: {} ", regionInfos); LOG.debug("Overwritten regions: {} ", regionInfos);
} }
/**
* Update region replicas if necessary by adding new replica locations or removing unused region
* replicas
*/
public void updateRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount)
throws IOException {
if (newReplicaCount < oldReplicaCount) {
removeRegionReplicas(tableName, oldReplicaCount, newReplicaCount);
} else if (newReplicaCount > oldReplicaCount) {
addRegionReplicas(tableName, oldReplicaCount, newReplicaCount);
}
}
private Scan getScanForUpdateRegionReplicas(TableName tableName) {
return MetaTableAccessor.getScanForTableName(master.getConfiguration(), tableName)
.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
}
private void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount)
throws IOException {
Scan scan = getScanForUpdateRegionReplicas(tableName);
List<Delete> deletes = new ArrayList<>();
long now = EnvironmentEdgeManager.currentTime();
try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) {
for (;;) {
Result result = scanner.next();
if (result == null) {
break;
}
RegionInfo primaryRegionInfo = CatalogFamilyFormat.getRegionInfo(result);
if (primaryRegionInfo == null || primaryRegionInfo.isSplitParent()) {
continue;
}
Delete delete = new Delete(result.getRow());
for (int i = newReplicaCount; i < oldReplicaCount; i++) {
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerColumn(i), now);
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getSeqNumColumn(i), now);
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getStartCodeColumn(i),
now);
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerNameColumn(i),
now);
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getRegionStateColumn(i),
now);
}
deletes.add(delete);
}
debugLogMutations(deletes);
metaTable.delete(deletes);
}
}
private void addRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount)
throws IOException {
Scan scan = getScanForUpdateRegionReplicas(tableName);
List<Put> puts = new ArrayList<>();
long now = EnvironmentEdgeManager.currentTime();
try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) {
for (;;) {
Result result = scanner.next();
if (result == null) {
break;
}
RegionInfo primaryRegionInfo = CatalogFamilyFormat.getRegionInfo(result);
if (primaryRegionInfo == null || primaryRegionInfo.isSplitParent()) {
continue;
}
Put put = new Put(result.getRow(), now);
for (int i = oldReplicaCount; i < newReplicaCount; i++) {
MetaTableAccessor.addEmptyLocation(put, i);
}
puts.add(put);
}
debugLogMutations(puts);
metaTable.put(puts);
}
}
// ========================================================================== // ==========================================================================
// Table Descriptors helpers // Table Descriptors helpers
// ========================================================================== // ==========================================================================

View File

@ -29,6 +29,7 @@ import java.util.Set;
import java.util.SortedSet; import java.util.SortedSet;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@ -190,8 +192,8 @@ public class MetaFixer {
// Add replicas if needed // Add replicas if needed
// we need to create regions with replicaIds starting from 1 // we need to create regions with replicaIds starting from 1
List<RegionInfo> newRegions = RegionReplicaUtil.addReplicas( List<RegionInfo> newRegions = RegionReplicaUtil
Collections.singletonList(regionInfo), 1, td.getRegionReplication()); .addReplicas(Collections.singletonList(regionInfo), 1, td.getRegionReplication());
// Add regions to META // Add regions to META
MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(), newRegions, MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(), newRegions,
@ -199,12 +201,13 @@ public class MetaFixer {
// Setup replication for region replicas if needed // Setup replication for region replicas if needed
if (td.getRegionReplication() > 1) { if (td.getRegionReplication() > 1) {
ServerRegionReplicaUtil.setupRegionReplicaReplication( ServerRegionReplicaUtil.setupRegionReplicaReplication(masterServices);
masterServices.getConfiguration());
} }
return Either.<List<RegionInfo>, IOException>ofLeft(newRegions); return Either.<List<RegionInfo>, IOException> ofLeft(newRegions);
} catch (IOException e) { } catch (IOException e) {
return Either.<List<RegionInfo>, IOException>ofRight(e); return Either.<List<RegionInfo>, IOException> ofRight(e);
} catch (ReplicationException e) {
return Either.<List<RegionInfo>, IOException> ofRight(new HBaseIOException(e));
} }
}) })
.collect(Collectors.toList()); .collect(Collectors.toList());

View File

@ -25,6 +25,7 @@ import java.util.function.Supplier;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
@ -363,23 +365,26 @@ public class CreateTableProcedure
} }
protected static List<RegionInfo> addTableToMeta(final MasterProcedureEnv env, protected static List<RegionInfo> addTableToMeta(final MasterProcedureEnv env,
final TableDescriptor tableDescriptor, final TableDescriptor tableDescriptor, final List<RegionInfo> regions) throws IOException {
final List<RegionInfo> regions) throws IOException {
assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions; assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
ProcedureSyncWait.waitMetaRegions(env); ProcedureSyncWait.waitMetaRegions(env);
// Add replicas if needed // Add replicas if needed
// we need to create regions with replicaIds starting from 1 // we need to create regions with replicaIds starting from 1
List<RegionInfo> newRegions = RegionReplicaUtil.addReplicas(regions, 1, List<RegionInfo> newRegions =
tableDescriptor.getRegionReplication()); RegionReplicaUtil.addReplicas(regions, 1, tableDescriptor.getRegionReplication());
// Add regions to META // Add regions to META
addRegionsToMeta(env, tableDescriptor, newRegions); addRegionsToMeta(env, tableDescriptor, newRegions);
// Setup replication for region replicas if needed // Setup replication for region replicas if needed
if (tableDescriptor.getRegionReplication() > 1) { if (tableDescriptor.getRegionReplication() > 1) {
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration()); try {
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterServices());
} catch (ReplicationException e) {
throw new HBaseIOException(e);
}
} }
return newRegions; return newRegions;
} }

View File

@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@ -40,12 +39,14 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.util.CommonFSUtils; 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.FSUtils;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -357,22 +358,29 @@ public class DeleteTableProcedure
/** /**
* There may be items for this table still up in hbase:meta in the case where the info:regioninfo * There may be items for this table still up in hbase:meta in the case where the info:regioninfo
* column was empty because of some write error. Remove ALL rows from hbase:meta that have to do * column was empty because of some write error. Remove ALL rows from hbase:meta that have to do
* with this table. See HBASE-12980. * with this table.
* <p/>
* See HBASE-12980.
*/ */
private static void cleanRegionsInMeta(final MasterProcedureEnv env, final TableName tableName) private static void cleanRegionsInMeta(final MasterProcedureEnv env, final TableName tableName)
throws IOException { throws IOException {
Connection connection = env.getMasterServices().getConnection(); Scan tableScan = MetaTableAccessor.getScanForTableName(env.getMasterConfiguration(), tableName)
Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName); .setFilter(new KeyOnlyFilter());
try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) { long now = EnvironmentEdgeManager.currentTime();
List<Delete> deletes = new ArrayList<>(); List<Delete> deletes = new ArrayList<>();
try (ResultScanner resScanner = metaTable.getScanner(tableScan)) { try (
for (Result result : resScanner) { Table metaTable = env.getMasterServices().getConnection().getTable(TableName.META_TABLE_NAME);
deletes.add(new Delete(result.getRow())); ResultScanner scanner = metaTable.getScanner(tableScan)) {
for (;;) {
Result result = scanner.next();
if (result == null) {
break;
} }
deletes.add(new Delete(result.getRow(), now));
} }
if (!deletes.isEmpty()) { if (!deletes.isEmpty()) {
LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from " LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from " +
+ TableName.META_TABLE_NAME); TableName.META_TABLE_NAME);
metaTable.delete(deletes); metaTable.delete(deletes);
} }
} }

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.function.Supplier; import java.util.function.Supplier;
@ -29,20 +28,15 @@ import org.apache.hadoop.hbase.ConcurrentTableModificationException;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@ -346,8 +340,6 @@ public class ModifyTableProcedure
* Action before modifying table. * Action before modifying table.
* @param env MasterProcedureEnv * @param env MasterProcedureEnv
* @param state the procedure state * @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/ */
private void preModify(final MasterProcedureEnv env, final ModifyTableState state) private void preModify(final MasterProcedureEnv env, final ModifyTableState state)
throws IOException, InterruptedException { throws IOException, InterruptedException {
@ -357,7 +349,6 @@ public class ModifyTableProcedure
/** /**
* Update descriptor * Update descriptor
* @param env MasterProcedureEnv * @param env MasterProcedureEnv
* @throws IOException
**/ **/
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException { private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor); env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor);
@ -366,7 +357,6 @@ public class ModifyTableProcedure
/** /**
* Removes from hdfs the families that are not longer present in the new table descriptor. * Removes from hdfs the families that are not longer present in the new table descriptor.
* @param env MasterProcedureEnv * @param env MasterProcedureEnv
* @throws IOException
*/ */
private void deleteFromFs(final MasterProcedureEnv env, private void deleteFromFs(final MasterProcedureEnv env,
final TableDescriptor oldTableDescriptor, final TableDescriptor newTableDescriptor) final TableDescriptor oldTableDescriptor, final TableDescriptor newTableDescriptor)
@ -386,61 +376,28 @@ public class ModifyTableProcedure
/** /**
* update replica column families if necessary. * update replica column families if necessary.
* @param env MasterProcedureEnv
* @throws IOException
*/ */
private void updateReplicaColumnsIfNeeded( private void updateReplicaColumnsIfNeeded(MasterProcedureEnv env,
final MasterProcedureEnv env, TableDescriptor oldTableDescriptor, TableDescriptor newTableDescriptor) throws IOException {
final TableDescriptor oldTableDescriptor,
final TableDescriptor newTableDescriptor) throws IOException {
final int oldReplicaCount = oldTableDescriptor.getRegionReplication(); final int oldReplicaCount = oldTableDescriptor.getRegionReplication();
final int newReplicaCount = newTableDescriptor.getRegionReplication(); final int newReplicaCount = newTableDescriptor.getRegionReplication();
env.getAssignmentManager().getRegionStateStore().updateRegionReplicas(getTableName(),
if (newReplicaCount < oldReplicaCount) { oldReplicaCount, newReplicaCount);
Set<byte[]> tableRows = new HashSet<>(); if (newReplicaCount > oldReplicaCount && oldReplicaCount <= 1) {
Connection connection = env.getMasterServices().getConnection();
Scan scan = MetaTableAccessor.getScanForTableName(connection, getTableName());
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) {
ResultScanner resScanner = metaTable.getScanner(scan);
for (Result result : resScanner) {
tableRows.add(result.getRow());
}
MetaTableAccessor.removeRegionReplicasFromMeta(
tableRows,
newReplicaCount,
oldReplicaCount - newReplicaCount,
connection);
}
}
if (newReplicaCount > oldReplicaCount) {
Connection connection = env.getMasterServices().getConnection();
// Get the existing table regions
List<RegionInfo> existingTableRegions =
MetaTableAccessor.getTableRegions(connection, getTableName());
// add all the new entries to the meta table
addRegionsToMeta(env, newTableDescriptor, existingTableRegions);
if (oldReplicaCount <= 1) {
// The table has been newly enabled for replica. So check if we need to setup // The table has been newly enabled for replica. So check if we need to setup
// region replication // region replication
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration()); try {
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterServices());
} catch (ReplicationException e) {
throw new HBaseIOException(e);
} }
} }
} }
private static void addRegionsToMeta(final MasterProcedureEnv env,
final TableDescriptor tableDescriptor, final List<RegionInfo> regionInfos)
throws IOException {
MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(), regionInfos,
tableDescriptor.getRegionReplication());
}
/** /**
* Action after modifying table. * Action after modifying table.
* @param env MasterProcedureEnv * @param env MasterProcedureEnv
* @param state the procedure state * @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/ */
private void postModify(final MasterProcedureEnv env, final ModifyTableState state) private void postModify(final MasterProcedureEnv env, final ModifyTableState state)
throws IOException, InterruptedException { throws IOException, InterruptedException {
@ -451,8 +408,6 @@ public class ModifyTableProcedure
* Coprocessor Action. * Coprocessor Action.
* @param env MasterProcedureEnv * @param env MasterProcedureEnv
* @param state the procedure state * @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/ */
private void runCoprocessorAction(final MasterProcedureEnv env, final ModifyTableState state) private void runCoprocessorAction(final MasterProcedureEnv env, final ModifyTableState state)
throws IOException, InterruptedException { throws IOException, InterruptedException {

View File

@ -22,16 +22,15 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint; import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint;
import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@ -155,34 +154,24 @@ public class ServerRegionReplicaUtil extends RegionReplicaUtil {
/** /**
* Create replication peer for replicating to region replicas if needed. * Create replication peer for replicating to region replicas if needed.
* @param conf configuration to use * <p/>
* @throws IOException * This methods should only be called at master side.
*/ */
public static void setupRegionReplicaReplication(Configuration conf) throws IOException { public static void setupRegionReplicaReplication(MasterServices services)
if (!isRegionReplicaReplicationEnabled(conf)) { throws IOException, ReplicationException {
if (!isRegionReplicaReplicationEnabled(services.getConfiguration())) {
return; return;
} }
if (services.getReplicationPeerManager().getPeerConfig(REGION_REPLICA_REPLICATION_PEER)
try (Connection connection = ConnectionFactory.createConnection(conf); .isPresent()) {
Admin admin = connection.getAdmin()) { return;
ReplicationPeerConfig peerConfig = null;
try {
peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
} catch (ReplicationPeerNotFoundException e) {
LOG.warn(
"Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER + " not exist",
e);
}
if (peerConfig == null) {
LOG.info("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
+ " not exist. Creating...");
peerConfig = new ReplicationPeerConfig();
peerConfig.setClusterKey(ZKConfig.getZooKeeperClusterKey(conf));
peerConfig.setReplicationEndpointImpl(RegionReplicaReplicationEndpoint.class.getName());
admin.addReplicationPeer(REGION_REPLICA_REPLICATION_PEER, peerConfig);
}
} }
LOG.info("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER +
" not exist. Creating...");
ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
.setClusterKey(ZKConfig.getZooKeeperClusterKey(services.getConfiguration()))
.setReplicationEndpointImpl(RegionReplicaReplicationEndpoint.class.getName()).build();
services.addReplicationPeer(REGION_REPLICA_REPLICATION_PEER, peerConfig, true);
} }
public static boolean isRegionReplicaReplicationEnabled(Configuration conf) { public static boolean isRegionReplicaReplicationEnabled(Configuration conf) {

View File

@ -2489,7 +2489,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
.setStartKey(startKeys[i]) .setStartKey(startKeys[i])
.setEndKey(startKeys[j]) .setEndKey(startKeys[j])
.build(); .build();
MetaTableAccessor.addRegionToMeta(getConnection(), hri); MetaTableAccessor.addRegionsToMeta(getConnection(), Collections.singletonList(hri), 1);
newRegions.add(hri); newRegions.add(hri);
} }

View File

@ -29,9 +29,8 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Random; import java.util.Random;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
@ -67,7 +66,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
/** /**
* Test {@link org.apache.hadoop.hbase.MetaTableAccessor}. * Test {@link org.apache.hadoop.hbase.MetaTableAccessor}.
@ -143,9 +141,11 @@ public class TestMetaTableAccessor {
} }
}; };
MetaTask writer = new MetaTask(connection, "writer") { MetaTask writer = new MetaTask(connection, "writer") {
@Override @Override
void metaTask() throws Throwable { void metaTask() throws IOException {
MetaTableAccessor.addRegionToMeta(connection, regions.get(0)); MetaTableAccessor.addRegionsToMeta(connection, Collections.singletonList(regions.get(0)),
1);
LOG.info("Wrote " + regions.get(0).getEncodedName()); LOG.info("Wrote " + regions.get(0).getEncodedName());
} }
}; };
@ -377,44 +377,6 @@ public class TestMetaTableAccessor {
assertEquals(0, startCodeCell.getValueLength()); assertEquals(0, startCodeCell.getValueLength());
} }
@Test
public void testMetaLocationForRegionReplicasIsRemovedAtTableDeletion() throws IOException {
long regionId = System.currentTimeMillis();
RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
.setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
.setRegionId(regionId).setReplicaId(0).build();
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
List<RegionInfo> regionInfos = Lists.newArrayList(primary);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
MetaTableAccessor.removeRegionReplicasFromMeta(Sets.newHashSet(primary.getRegionName()), 1, 2,
connection);
Get get = new Get(primary.getRegionName());
Result result = meta.get(get);
for (int replicaId = 0; replicaId < 3; replicaId++) {
Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getServerColumn(replicaId));
Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getStartCodeColumn(replicaId));
Cell stateCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getRegionStateColumn(replicaId));
Cell snCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getServerNameColumn(replicaId));
if (replicaId == 0) {
assertNotNull(stateCell);
} else {
assertNull(serverCell);
assertNull(startCodeCell);
assertNull(stateCell);
assertNull(snCell);
}
}
} finally {
meta.close();
}
}
@Test @Test
public void testMetaLocationForRegionReplicasIsAddedAtTableCreation() throws IOException { public void testMetaLocationForRegionReplicasIsAddedAtTableCreation() throws IOException {
long regionId = System.currentTimeMillis(); long regionId = System.currentTimeMillis();
@ -434,14 +396,6 @@ public class TestMetaTableAccessor {
} }
} }
private Map<RegionInfo, Long> getMapOfRegionsToSeqNum(RegionInfo... regions) {
Map<RegionInfo, Long> mids = new HashMap<>(regions.length);
for (RegionInfo region : regions) {
mids.put(region, -1L);
}
return mids;
}
@Test @Test
public void testMetaScanner() throws Exception { public void testMetaScanner() throws Exception {
LOG.info("Starting " + name.getMethodName()); LOG.info("Starting " + name.getMethodName());
@ -454,7 +408,7 @@ public class TestMetaTableAccessor {
UTIL.createTable(tableName, FAMILY, SPLIT_KEYS); UTIL.createTable(tableName, FAMILY, SPLIT_KEYS);
Table table = connection.getTable(tableName); Table table = connection.getTable(tableName);
// Make sure all the regions are deployed // Make sure all the regions are deployed
UTIL.countRows(table); HBaseTestingUtility.countRows(table);
ClientMetaTableAccessor.Visitor visitor = mock(ClientMetaTableAccessor.Visitor.class); ClientMetaTableAccessor.Visitor visitor = mock(ClientMetaTableAccessor.Visitor.class);
doReturn(true).when(visitor).visit((Result) anyObject()); doReturn(true).when(visitor).visit((Result) anyObject());

View File

@ -94,7 +94,7 @@ public class TestEnableTable {
// content from a few of the rows. // content from a few of the rows.
try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
try (ResultScanner scanner = metaTable.getScanner( try (ResultScanner scanner = metaTable.getScanner(
MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName))) { MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) {
for (Result result : scanner) { for (Result result : scanner) {
// Just delete one row. // Just delete one row.
Delete d = new Delete(result.getRow()); Delete d = new Delete(result.getRow());
@ -114,8 +114,8 @@ public class TestEnableTable {
fail("Got an exception while deleting " + tableName); fail("Got an exception while deleting " + tableName);
} }
int rowCount = 0; int rowCount = 0;
try (ResultScanner scanner = metaTable try (ResultScanner scanner = metaTable.getScanner(
.getScanner(MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName))) { MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) {
for (Result result : scanner) { for (Result result : scanner) {
LOG.info("Found when none expected: " + result); LOG.info("Found when none expected: " + result);
rowCount++; rowCount++;

View File

@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.util.Collections;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
@ -299,7 +300,8 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
RegionInfo hri = createRegionInfo(tableName, 1); RegionInfo hri = createRegionInfo(tableName, 1);
assertNull("RegionInfo was just instantiated by the test, but " assertNull("RegionInfo was just instantiated by the test, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri)); + "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
MetaTableAccessor.addRegionToMeta(this.util.getConnection(), hri); MetaTableAccessor.addRegionsToMeta(this.util.getConnection(), Collections.singletonList(hri),
1);
assertNull("RegionInfo was manually added in META, but " assertNull("RegionInfo was manually added in META, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri)); + "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
hri = am.loadRegionFromMeta(hri.getEncodedName()); hri = am.loadRegionFromMeta(hri.getEncodedName());

View File

@ -394,4 +394,41 @@ public class TestRegionStateStore {
previousQualifier = qualifier; previousQualifier = qualifier;
} }
} }
@Test
public void testMetaLocationForRegionReplicasIsRemovedAtTableDeletion() throws IOException {
long regionId = System.currentTimeMillis();
TableName tableName = name.getTableName();
RegionInfo primary = RegionInfoBuilder.newBuilder(tableName)
.setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
.setRegionId(regionId).setReplicaId(0).build();
try (Table meta = MetaTableAccessor.getMetaHTable(UTIL.getConnection())) {
List<RegionInfo> regionInfos = Lists.newArrayList(primary);
MetaTableAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
final RegionStateStore regionStateStore =
UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
regionStateStore.updateRegionReplicas(tableName, 3, 1);
Get get = new Get(primary.getRegionName());
Result result = meta.get(get);
for (int replicaId = 0; replicaId < 3; replicaId++) {
Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getServerColumn(replicaId));
Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getStartCodeColumn(replicaId));
Cell stateCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getRegionStateColumn(replicaId));
Cell snCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
CatalogFamilyFormat.getServerNameColumn(replicaId));
if (replicaId == 0) {
assertNotNull(stateCell);
} else {
assertNull(serverCell);
assertNull(startCodeCell);
assertNull(stateCell);
assertNull(snCell);
}
}
}
}
} }