HBASE-25121 Refactor MetaTableAccessor.addRegionsToMeta and its usage places (#2476)
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
23ce91819a
commit
9ba90e1679
|
@ -27,7 +27,7 @@ import java.util.LinkedHashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
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.ClientMetaTableAccessor.QueryType;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
|
@ -247,7 +247,7 @@ public final class MetaTableAccessor {
|
|||
throws IOException {
|
||||
RowFilter rowFilter =
|
||||
new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
|
||||
Scan scan = getMetaScan(connection, 1);
|
||||
Scan scan = getMetaScan(connection.getConfiguration(), 1);
|
||||
scan.setFilter(rowFilter);
|
||||
try (Table table = getMetaHTable(connection);
|
||||
ResultScanner resultScanner = table.getScanner(scan)) {
|
||||
|
@ -320,24 +320,23 @@ public final class MetaTableAccessor {
|
|||
* @param tableName bytes of table's name
|
||||
* @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
|
||||
byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
|
||||
// Stop key appends the smallest possible char to the table name
|
||||
byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
|
||||
|
||||
Scan scan = getMetaScan(connection, -1);
|
||||
Scan scan = getMetaScan(conf, -1);
|
||||
scan.withStartRow(startKey);
|
||||
scan.withStopRow(stopKey);
|
||||
return scan;
|
||||
}
|
||||
|
||||
private static Scan getMetaScan(Connection connection, int rowUpperLimit) {
|
||||
private static Scan getMetaScan(Configuration conf, int rowUpperLimit) {
|
||||
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);
|
||||
if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS)) {
|
||||
if (conf.getBoolean(HConstants.USE_META_REPLICAS, HConstants.DEFAULT_USE_META_REPLICAS)) {
|
||||
scan.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
if (rowUpperLimit > 0) {
|
||||
|
@ -469,7 +468,7 @@ public final class MetaTableAccessor {
|
|||
@Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
|
||||
final ClientMetaTableAccessor.Visitor visitor) throws IOException {
|
||||
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
|
||||
Scan scan = getMetaScan(connection, rowUpperLimit);
|
||||
Scan scan = getMetaScan(connection.getConfiguration(), rowUpperLimit);
|
||||
|
||||
for (byte[] family : type.getFamilies()) {
|
||||
scan.addFamily(family);
|
||||
|
@ -525,7 +524,7 @@ public final class MetaTableAccessor {
|
|||
private static RegionInfo getClosestRegionInfo(Connection connection,
|
||||
@NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
|
||||
byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
|
||||
Scan scan = getMetaScan(connection, 1);
|
||||
Scan scan = getMetaScan(connection.getConfiguration(), 1);
|
||||
scan.setReversed(true);
|
||||
scan.withStartRow(searchRow);
|
||||
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 {
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.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
|
||||
* CLOSED.
|
||||
|
@ -845,17 +797,18 @@ public final class MetaTableAccessor {
|
|||
int regionReplication, long ts) throws IOException {
|
||||
List<Put> puts = new ArrayList<>();
|
||||
for (RegionInfo regionInfo : regionInfos) {
|
||||
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
|
||||
Put put = makePutFromRegionInfo(regionInfo, ts);
|
||||
// New regions are added with initial state of CLOSED.
|
||||
addRegionStateToPut(put, RegionState.State.CLOSED);
|
||||
// Add empty locations for region replicas so that number of replicas can be cached
|
||||
// whenever the primary region is looked up from meta
|
||||
for (int i = 1; i < regionReplication; i++) {
|
||||
addEmptyLocation(put, i);
|
||||
}
|
||||
puts.add(put);
|
||||
if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
|
||||
continue;
|
||||
}
|
||||
Put put = makePutFromRegionInfo(regionInfo, ts);
|
||||
// New regions are added with initial state of CLOSED.
|
||||
addRegionStateToPut(put, RegionState.State.CLOSED);
|
||||
// Add empty locations for region replicas so that number of replicas can be cached
|
||||
// whenever the primary region is looked up from meta
|
||||
for (int i = 1; i < regionReplication; i++) {
|
||||
addEmptyLocation(put, i);
|
||||
}
|
||||
puts.add(put);
|
||||
}
|
||||
putsToMetaTable(connection, puts);
|
||||
LOG.info("Added {} regions to meta.", puts.size());
|
||||
|
|
|
@ -311,8 +311,7 @@ public class CatalogFamilyFormat {
|
|||
* @param replicaId the replicaId of the region
|
||||
* @return a byte[] for state qualifier
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static byte[] getRegionStateColumn(int replicaId) {
|
||||
public static byte[] getRegionStateColumn(int replicaId) {
|
||||
return replicaId == 0 ? HConstants.STATE_QUALIFIER :
|
||||
Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER +
|
||||
String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
|
||||
|
|
|
@ -48,6 +48,8 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
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.master.MasterFileSystem;
|
||||
|
@ -545,6 +547,83 @@ public class RegionStateStore {
|
|||
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
|
||||
// ==========================================================================
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.Set;
|
|||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
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.master.MasterServices;
|
||||
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.Pair;
|
||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||
|
@ -190,8 +192,8 @@ public class MetaFixer {
|
|||
|
||||
// Add replicas if needed
|
||||
// we need to create regions with replicaIds starting from 1
|
||||
List<RegionInfo> newRegions = RegionReplicaUtil.addReplicas(
|
||||
Collections.singletonList(regionInfo), 1, td.getRegionReplication());
|
||||
List<RegionInfo> newRegions = RegionReplicaUtil
|
||||
.addReplicas(Collections.singletonList(regionInfo), 1, td.getRegionReplication());
|
||||
|
||||
// Add regions to META
|
||||
MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(), newRegions,
|
||||
|
@ -199,12 +201,13 @@ public class MetaFixer {
|
|||
|
||||
// Setup replication for region replicas if needed
|
||||
if (td.getRegionReplication() > 1) {
|
||||
ServerRegionReplicaUtil.setupRegionReplicaReplication(
|
||||
masterServices.getConfiguration());
|
||||
ServerRegionReplicaUtil.setupRegionReplicaReplication(masterServices);
|
||||
}
|
||||
return Either.<List<RegionInfo>, IOException>ofLeft(newRegions);
|
||||
return Either.<List<RegionInfo>, IOException> ofLeft(newRegions);
|
||||
} 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());
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.function.Supplier;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
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.MasterFileSystem;
|
||||
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.util.CommonFSUtils;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
|
@ -363,23 +365,26 @@ public class CreateTableProcedure
|
|||
}
|
||||
|
||||
protected static List<RegionInfo> addTableToMeta(final MasterProcedureEnv env,
|
||||
final TableDescriptor tableDescriptor,
|
||||
final List<RegionInfo> regions) throws IOException {
|
||||
final TableDescriptor tableDescriptor, final List<RegionInfo> regions) throws IOException {
|
||||
assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
|
||||
|
||||
ProcedureSyncWait.waitMetaRegions(env);
|
||||
|
||||
// Add replicas if needed
|
||||
// we need to create regions with replicaIds starting from 1
|
||||
List<RegionInfo> newRegions = RegionReplicaUtil.addReplicas(regions, 1,
|
||||
tableDescriptor.getRegionReplication());
|
||||
List<RegionInfo> newRegions =
|
||||
RegionReplicaUtil.addReplicas(regions, 1, tableDescriptor.getRegionReplication());
|
||||
|
||||
// Add regions to META
|
||||
addRegionsToMeta(env, tableDescriptor, newRegions);
|
||||
|
||||
// Setup replication for region replicas if needed
|
||||
if (tableDescriptor.getRegionReplication() > 1) {
|
||||
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
|
||||
try {
|
||||
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterServices());
|
||||
} catch (ReplicationException e) {
|
||||
throw new HBaseIOException(e);
|
||||
}
|
||||
}
|
||||
return newRegions;
|
||||
}
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
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.RegionInfo;
|
||||
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.Table;
|
||||
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.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
import org.apache.hadoop.hbase.util.CommonFSUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
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
|
||||
* 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)
|
||||
throws IOException {
|
||||
Connection connection = env.getMasterServices().getConnection();
|
||||
Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
|
||||
try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) {
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
try (ResultScanner resScanner = metaTable.getScanner(tableScan)) {
|
||||
for (Result result : resScanner) {
|
||||
deletes.add(new Delete(result.getRow()));
|
||||
throws IOException {
|
||||
Scan tableScan = MetaTableAccessor.getScanForTableName(env.getMasterConfiguration(), tableName)
|
||||
.setFilter(new KeyOnlyFilter());
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
try (
|
||||
Table metaTable = env.getMasterServices().getConnection().getTable(TableName.META_TABLE_NAME);
|
||||
ResultScanner scanner = metaTable.getScanner(tableScan)) {
|
||||
for (;;) {
|
||||
Result result = scanner.next();
|
||||
if (result == null) {
|
||||
break;
|
||||
}
|
||||
deletes.add(new Delete(result.getRow(), now));
|
||||
}
|
||||
if (!deletes.isEmpty()) {
|
||||
LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from "
|
||||
+ TableName.META_TABLE_NAME);
|
||||
LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from " +
|
||||
TableName.META_TABLE_NAME);
|
||||
metaTable.delete(deletes);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master.procedure;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
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.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||
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.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.TableState;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
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.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||
|
@ -346,8 +340,6 @@ public class ModifyTableProcedure
|
|||
* Action before modifying table.
|
||||
* @param env MasterProcedureEnv
|
||||
* @param state the procedure state
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private void preModify(final MasterProcedureEnv env, final ModifyTableState state)
|
||||
throws IOException, InterruptedException {
|
||||
|
@ -357,7 +349,6 @@ public class ModifyTableProcedure
|
|||
/**
|
||||
* Update descriptor
|
||||
* @param env MasterProcedureEnv
|
||||
* @throws IOException
|
||||
**/
|
||||
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
|
||||
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.
|
||||
* @param env MasterProcedureEnv
|
||||
* @throws IOException
|
||||
*/
|
||||
private void deleteFromFs(final MasterProcedureEnv env,
|
||||
final TableDescriptor oldTableDescriptor, final TableDescriptor newTableDescriptor)
|
||||
|
@ -386,61 +376,28 @@ public class ModifyTableProcedure
|
|||
|
||||
/**
|
||||
* update replica column families if necessary.
|
||||
* @param env MasterProcedureEnv
|
||||
* @throws IOException
|
||||
*/
|
||||
private void updateReplicaColumnsIfNeeded(
|
||||
final MasterProcedureEnv env,
|
||||
final TableDescriptor oldTableDescriptor,
|
||||
final TableDescriptor newTableDescriptor) throws IOException {
|
||||
private void updateReplicaColumnsIfNeeded(MasterProcedureEnv env,
|
||||
TableDescriptor oldTableDescriptor, TableDescriptor newTableDescriptor) throws IOException {
|
||||
final int oldReplicaCount = oldTableDescriptor.getRegionReplication();
|
||||
final int newReplicaCount = newTableDescriptor.getRegionReplication();
|
||||
|
||||
if (newReplicaCount < oldReplicaCount) {
|
||||
Set<byte[]> tableRows = new HashSet<>();
|
||||
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
|
||||
// region replication
|
||||
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
|
||||
env.getAssignmentManager().getRegionStateStore().updateRegionReplicas(getTableName(),
|
||||
oldReplicaCount, newReplicaCount);
|
||||
if (newReplicaCount > oldReplicaCount && oldReplicaCount <= 1) {
|
||||
// The table has been newly enabled for replica. So check if we need to setup
|
||||
// region replication
|
||||
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.
|
||||
* @param env MasterProcedureEnv
|
||||
* @param state the procedure state
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private void postModify(final MasterProcedureEnv env, final ModifyTableState state)
|
||||
throws IOException, InterruptedException {
|
||||
|
@ -451,8 +408,6 @@ public class ModifyTableProcedure
|
|||
* Coprocessor Action.
|
||||
* @param env MasterProcedureEnv
|
||||
* @param state the procedure state
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private void runCoprocessorAction(final MasterProcedureEnv env, final ModifyTableState state)
|
||||
throws IOException, InterruptedException {
|
||||
|
|
|
@ -22,16 +22,15 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
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.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationException;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint;
|
||||
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.
|
||||
* @param conf configuration to use
|
||||
* @throws IOException
|
||||
* <p/>
|
||||
* This methods should only be called at master side.
|
||||
*/
|
||||
public static void setupRegionReplicaReplication(Configuration conf) throws IOException {
|
||||
if (!isRegionReplicaReplicationEnabled(conf)) {
|
||||
public static void setupRegionReplicaReplication(MasterServices services)
|
||||
throws IOException, ReplicationException {
|
||||
if (!isRegionReplicaReplicationEnabled(services.getConfiguration())) {
|
||||
return;
|
||||
}
|
||||
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
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);
|
||||
}
|
||||
if (services.getReplicationPeerManager().getPeerConfig(REGION_REPLICA_REPLICATION_PEER)
|
||||
.isPresent()) {
|
||||
return;
|
||||
}
|
||||
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) {
|
||||
|
|
|
@ -2489,7 +2489,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
|
|||
.setStartKey(startKeys[i])
|
||||
.setEndKey(startKeys[j])
|
||||
.build();
|
||||
MetaTableAccessor.addRegionToMeta(getConnection(), hri);
|
||||
MetaTableAccessor.addRegionsToMeta(getConnection(), Collections.singletonList(hri), 1);
|
||||
newRegions.add(hri);
|
||||
}
|
||||
|
||||
|
|
|
@ -29,9 +29,8 @@ import static org.mockito.Mockito.times;
|
|||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
|
@ -67,7 +66,6 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
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}.
|
||||
|
@ -143,9 +141,11 @@ public class TestMetaTableAccessor {
|
|||
}
|
||||
};
|
||||
MetaTask writer = new MetaTask(connection, "writer") {
|
||||
|
||||
@Override
|
||||
void metaTask() throws Throwable {
|
||||
MetaTableAccessor.addRegionToMeta(connection, regions.get(0));
|
||||
void metaTask() throws IOException {
|
||||
MetaTableAccessor.addRegionsToMeta(connection, Collections.singletonList(regions.get(0)),
|
||||
1);
|
||||
LOG.info("Wrote " + regions.get(0).getEncodedName());
|
||||
}
|
||||
};
|
||||
|
@ -377,44 +377,6 @@ public class TestMetaTableAccessor {
|
|||
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
|
||||
public void testMetaLocationForRegionReplicasIsAddedAtTableCreation() throws IOException {
|
||||
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
|
||||
public void testMetaScanner() throws Exception {
|
||||
LOG.info("Starting " + name.getMethodName());
|
||||
|
@ -454,7 +408,7 @@ public class TestMetaTableAccessor {
|
|||
UTIL.createTable(tableName, FAMILY, SPLIT_KEYS);
|
||||
Table table = connection.getTable(tableName);
|
||||
// Make sure all the regions are deployed
|
||||
UTIL.countRows(table);
|
||||
HBaseTestingUtility.countRows(table);
|
||||
|
||||
ClientMetaTableAccessor.Visitor visitor = mock(ClientMetaTableAccessor.Visitor.class);
|
||||
doReturn(true).when(visitor).visit((Result) anyObject());
|
||||
|
|
|
@ -94,7 +94,7 @@ public class TestEnableTable {
|
|||
// content from a few of the rows.
|
||||
try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
|
||||
try (ResultScanner scanner = metaTable.getScanner(
|
||||
MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName))) {
|
||||
MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) {
|
||||
for (Result result : scanner) {
|
||||
// Just delete one row.
|
||||
Delete d = new Delete(result.getRow());
|
||||
|
@ -114,8 +114,8 @@ public class TestEnableTable {
|
|||
fail("Got an exception while deleting " + tableName);
|
||||
}
|
||||
int rowCount = 0;
|
||||
try (ResultScanner scanner = metaTable
|
||||
.getScanner(MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName))) {
|
||||
try (ResultScanner scanner = metaTable.getScanner(
|
||||
MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) {
|
||||
for (Result result : scanner) {
|
||||
LOG.info("Found when none expected: " + result);
|
||||
rowCount++;
|
||||
|
|
|
@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
|
@ -299,7 +300,8 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
|
|||
RegionInfo hri = createRegionInfo(tableName, 1);
|
||||
assertNull("RegionInfo was just instantiated by the test, but "
|
||||
+ "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 "
|
||||
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
|
||||
hri = am.loadRegionFromMeta(hri.getEncodedName());
|
||||
|
|
|
@ -394,4 +394,41 @@ public class TestRegionStateStore {
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue