001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master.assignment; 019 020import java.io.IOException; 021import java.util.ArrayList; 022import java.util.Collections; 023import java.util.List; 024import java.util.SortedMap; 025import java.util.TreeMap; 026import org.apache.hadoop.hbase.Cell; 027import org.apache.hadoop.hbase.CellBuilderFactory; 028import org.apache.hadoop.hbase.CellBuilderType; 029import org.apache.hadoop.hbase.HConstants; 030import org.apache.hadoop.hbase.HRegionLocation; 031import org.apache.hadoop.hbase.MetaTableAccessor; 032import org.apache.hadoop.hbase.RegionLocations; 033import org.apache.hadoop.hbase.ServerName; 034import org.apache.hadoop.hbase.TableName; 035import org.apache.hadoop.hbase.client.Delete; 036import org.apache.hadoop.hbase.client.Mutation; 037import org.apache.hadoop.hbase.client.Put; 038import org.apache.hadoop.hbase.client.RegionInfo; 039import org.apache.hadoop.hbase.client.Result; 040import org.apache.hadoop.hbase.client.ResultScanner; 041import org.apache.hadoop.hbase.client.Scan; 042import org.apache.hadoop.hbase.client.Table; 043import org.apache.hadoop.hbase.client.TableDescriptor; 044import org.apache.hadoop.hbase.master.MasterFileSystem; 045import org.apache.hadoop.hbase.master.MasterServices; 046import org.apache.hadoop.hbase.master.RegionState.State; 047import org.apache.hadoop.hbase.master.region.MasterRegion; 048import org.apache.hadoop.hbase.procedure2.Procedure; 049import org.apache.hadoop.hbase.procedure2.util.StringUtils; 050import org.apache.hadoop.hbase.util.Bytes; 051import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 052import org.apache.hadoop.hbase.wal.WALSplitUtil; 053import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; 054import org.apache.yetus.audience.InterfaceAudience; 055import org.apache.zookeeper.KeeperException; 056import org.slf4j.Logger; 057import org.slf4j.LoggerFactory; 058 059import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 060 061/** 062 * Store Region State to hbase:meta table. 063 */ 064@InterfaceAudience.Private 065public class RegionStateStore { 066 private static final Logger LOG = LoggerFactory.getLogger(RegionStateStore.class); 067 private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META"); 068 069 /** The delimiter for meta columns for replicaIds > 0 */ 070 protected static final char META_REPLICA_ID_DELIMITER = '_'; 071 072 private final MasterServices master; 073 074 private final MasterRegion masterRegion; 075 076 public RegionStateStore(MasterServices master, MasterRegion masterRegion) { 077 this.master = master; 078 this.masterRegion = masterRegion; 079 } 080 081 @FunctionalInterface 082 public interface RegionStateVisitor { 083 void visitRegionState(Result result, RegionInfo regionInfo, State state, 084 ServerName regionLocation, ServerName lastHost, long openSeqNum); 085 } 086 087 public void visitMeta(final RegionStateVisitor visitor) throws IOException { 088 MetaTableAccessor.fullScanRegions(master.getConnection(), new MetaTableAccessor.Visitor() { 089 final boolean isDebugEnabled = LOG.isDebugEnabled(); 090 091 @Override 092 public boolean visit(final Result r) throws IOException { 093 if (r != null && !r.isEmpty()) { 094 long st = 0; 095 if (LOG.isTraceEnabled()) { 096 st = EnvironmentEdgeManager.currentTime(); 097 } 098 visitMetaEntry(visitor, r); 099 if (LOG.isTraceEnabled()) { 100 long et = EnvironmentEdgeManager.currentTime(); 101 LOG.trace("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st)); 102 } 103 } else if (isDebugEnabled) { 104 LOG.debug("NULL result from meta - ignoring but this is strange."); 105 } 106 return true; 107 } 108 }); 109 } 110 111 /** 112 * Queries META table for the passed region encoded name, delegating action upon results to the 113 * {@code RegionStateVisitor} passed as second parameter. 114 * @param regionEncodedName encoded name for the Region we want to query META for. 115 * @param visitor The {@code RegionStateVisitor} instance to react over the query 116 * results. 117 * @throws IOException If some error occurs while querying META or parsing results. 118 */ 119 public void visitMetaForRegion(final String regionEncodedName, final RegionStateVisitor visitor) 120 throws IOException { 121 Result result = 122 MetaTableAccessor.scanByRegionEncodedName(master.getConnection(), regionEncodedName); 123 if (result != null) { 124 visitMetaEntry(visitor, result); 125 } 126 } 127 128 public static void visitMetaEntry(final RegionStateVisitor visitor, final Result result) 129 throws IOException { 130 final RegionLocations rl = MetaTableAccessor.getRegionLocations(result); 131 if (rl == null) return; 132 133 final HRegionLocation[] locations = rl.getRegionLocations(); 134 if (locations == null) return; 135 136 for (int i = 0; i < locations.length; ++i) { 137 final HRegionLocation hrl = locations[i]; 138 if (hrl == null) continue; 139 140 final RegionInfo regionInfo = hrl.getRegion(); 141 if (regionInfo == null) continue; 142 143 final int replicaId = regionInfo.getReplicaId(); 144 final State state = getRegionState(result, regionInfo); 145 146 final ServerName lastHost = hrl.getServerName(); 147 ServerName regionLocation = MetaTableAccessor.getTargetServerName(result, replicaId); 148 final long openSeqNum = hrl.getSeqNum(); 149 150 LOG.debug( 151 "Load hbase:meta entry region={}, regionState={}, lastHost={}, " 152 + "regionLocation={}, openSeqNum={}", 153 regionInfo.getEncodedName(), state, lastHost, regionLocation, openSeqNum); 154 visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum); 155 } 156 } 157 158 void updateRegionLocation(RegionStateNode regionStateNode) throws IOException { 159 long time = EnvironmentEdgeManager.currentTime(); 160 long openSeqNum = regionStateNode.getState() == State.OPEN 161 ? regionStateNode.getOpenSeqNum() 162 : HConstants.NO_SEQNUM; 163 RegionInfo regionInfo = regionStateNode.getRegionInfo(); 164 State state = regionStateNode.getState(); 165 ServerName regionLocation = regionStateNode.getRegionLocation(); 166 TransitRegionStateProcedure rit = regionStateNode.getProcedure(); 167 long pid = rit != null ? rit.getProcId() : Procedure.NO_PROC_ID; 168 final int replicaId = regionInfo.getReplicaId(); 169 final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo), time); 170 MetaTableAccessor.addRegionInfo(put, regionInfo); 171 final StringBuilder info = 172 new StringBuilder("pid=").append(pid).append(" updating hbase:meta row=") 173 .append(regionInfo.getEncodedName()).append(", regionState=").append(state); 174 if (openSeqNum >= 0) { 175 Preconditions.checkArgument(state == State.OPEN && regionLocation != null, 176 "Open region should be on a server"); 177 MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId); 178 // only update replication barrier for default replica 179 if ( 180 regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID 181 && hasGlobalReplicationScope(regionInfo.getTable()) 182 ) { 183 MetaTableAccessor.addReplicationBarrier(put, openSeqNum); 184 info.append(", repBarrier=").append(openSeqNum); 185 } 186 info.append(", openSeqNum=").append(openSeqNum); 187 info.append(", regionLocation=").append(regionLocation); 188 } else if (regionLocation != null) { 189 // Ideally, if no regionLocation, write null to the hbase:meta but this will confuse clients 190 // currently; they want a server to hit. TODO: Make clients wait if no location. 191 put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow()) 192 .setFamily(HConstants.CATALOG_FAMILY) 193 .setQualifier(MetaTableAccessor.getServerNameColumn(replicaId)) 194 .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put) 195 .setValue(Bytes.toBytes(regionLocation.getServerName())).build()); 196 info.append(", regionLocation=").append(regionLocation); 197 } 198 put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow()) 199 .setFamily(HConstants.CATALOG_FAMILY).setQualifier(getStateColumn(replicaId)) 200 .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name())) 201 .build()); 202 LOG.info(info.toString()); 203 updateRegionLocation(regionInfo, state, put); 204 if (regionInfo.isMetaRegion() && regionInfo.isFirst()) { 205 // mirror the meta location to zookeeper 206 mirrorMetaLocation(regionInfo, regionLocation, state); 207 } 208 } 209 210 private void mirrorMetaLocation(RegionInfo regionInfo, ServerName serverName, State state) 211 throws IOException { 212 try { 213 MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(), 214 state); 215 } catch (KeeperException e) { 216 throw new IOException(e); 217 } 218 } 219 220 private void removeMirrorMetaLocation(int oldReplicaCount, int newReplicaCount) 221 throws IOException { 222 try { 223 for (int i = newReplicaCount; i < oldReplicaCount; i++) { 224 MetaTableLocator.deleteMetaLocation(master.getZooKeeper(), i); 225 } 226 } catch (KeeperException e) { 227 throw new IOException(e); 228 } 229 } 230 231 private void updateRegionLocation(RegionInfo regionInfo, State state, Put put) 232 throws IOException { 233 try { 234 if (regionInfo.isMetaRegion()) { 235 masterRegion.update(r -> r.put(put)); 236 } else { 237 try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) { 238 table.put(put); 239 } 240 } 241 } catch (IOException e) { 242 // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host! 243 // In tests we abort the Master! 244 String msg = String.format("FAILED persisting region=%s state=%s", 245 regionInfo.getShortNameToLog(), state); 246 LOG.error(msg, e); 247 master.abort(msg, e); 248 throw e; 249 } 250 } 251 252 private long getOpenSeqNumForParentRegion(RegionInfo region) throws IOException { 253 MasterFileSystem fs = master.getMasterFileSystem(); 254 long maxSeqId = WALSplitUtil.getMaxRegionSequenceId(master.getConfiguration(), region, 255 fs::getFileSystem, fs::getWALFileSystem); 256 return maxSeqId > 0 ? maxSeqId + 1 : HConstants.NO_SEQNUM; 257 } 258 259 private Table getMetaTable() throws IOException { 260 return master.getConnection().getTable(TableName.META_TABLE_NAME); 261 } 262 263 // ============================================================================================ 264 // Update Region Splitting State helpers 265 // ============================================================================================ 266 public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB, 267 ServerName serverName) throws IOException { 268 TableDescriptor htd = getTableDescriptor(parent.getTable()); 269 long parentOpenSeqNum = HConstants.NO_SEQNUM; 270 if (htd.hasGlobalReplicationScope()) { 271 parentOpenSeqNum = getOpenSeqNumForParentRegion(parent); 272 } 273 MetaTableAccessor.splitRegion(master.getConnection(), parent, parentOpenSeqNum, hriA, hriB, 274 serverName, getRegionReplication(htd)); 275 } 276 277 // ============================================================================================ 278 // Update Region Merging State helpers 279 // ============================================================================================ 280 public void mergeRegions(RegionInfo child, RegionInfo[] parents, ServerName serverName) 281 throws IOException { 282 TableDescriptor htd = getTableDescriptor(child.getTable()); 283 boolean globalScope = htd.hasGlobalReplicationScope(); 284 SortedMap<RegionInfo, Long> parentSeqNums = new TreeMap<>(); 285 for (RegionInfo ri : parents) { 286 parentSeqNums.put(ri, globalScope ? getOpenSeqNumForParentRegion(ri) : -1); 287 } 288 MetaTableAccessor.mergeRegions(master.getConnection(), child, parentSeqNums, serverName, 289 getRegionReplication(htd)); 290 } 291 292 // ============================================================================================ 293 // Delete Region State helpers 294 // ============================================================================================ 295 public void deleteRegion(final RegionInfo regionInfo) throws IOException { 296 deleteRegions(Collections.singletonList(regionInfo)); 297 } 298 299 public void deleteRegions(final List<RegionInfo> regions) throws IOException { 300 MetaTableAccessor.deleteRegionInfos(master.getConnection(), regions); 301 } 302 303 private Scan getScanForUpdateRegionReplicas(TableName tableName) { 304 Scan scan; 305 if (TableName.isMetaTableName(tableName)) { 306 // Notice that, we do not use MetaCellComparator for master local region, so we can not use 307 // the same logic to set start key and end key for scanning meta table when locating entries 308 // in master local region. And since there is only one table in master local region(the record 309 // for meta table), so we do not need set start key and end key. 310 scan = new Scan(); 311 } else { 312 scan = MetaTableAccessor.getScanForTableName(master.getConfiguration(), tableName); 313 } 314 return scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); 315 } 316 317 private List<Delete> deleteRegionReplicas(ResultScanner scanner, int oldReplicaCount, 318 int newReplicaCount, long now) throws IOException { 319 List<Delete> deletes = new ArrayList<>(); 320 for (;;) { 321 Result result = scanner.next(); 322 if (result == null) { 323 break; 324 } 325 RegionInfo primaryRegionInfo = MetaTableAccessor.getRegionInfo(result); 326 if (primaryRegionInfo == null || primaryRegionInfo.isSplit()) { 327 continue; 328 } 329 Delete delete = new Delete(result.getRow()); 330 for (int i = newReplicaCount; i < oldReplicaCount; i++) { 331 delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i), now); 332 delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(i), now); 333 delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i), now); 334 delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerNameColumn(i), now); 335 delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getRegionStateColumn(i), 336 now); 337 } 338 deletes.add(delete); 339 } 340 return deletes; 341 } 342 343 public void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount) 344 throws IOException { 345 Scan scan = getScanForUpdateRegionReplicas(tableName); 346 long now = EnvironmentEdgeManager.currentTime(); 347 if (TableName.isMetaTableName(tableName)) { 348 List<Delete> deletes; 349 try (ResultScanner scanner = masterRegion.getScanner(scan)) { 350 deletes = deleteRegionReplicas(scanner, oldReplicaCount, newReplicaCount, now); 351 } 352 debugLogMutations(deletes); 353 masterRegion.update(r -> { 354 for (Delete d : deletes) { 355 r.delete(d); 356 } 357 }); 358 // also delete the mirrored location on zk 359 removeMirrorMetaLocation(oldReplicaCount, newReplicaCount); 360 } else { 361 try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) { 362 List<Delete> deletes = deleteRegionReplicas(scanner, oldReplicaCount, newReplicaCount, now); 363 debugLogMutations(deletes); 364 metaTable.delete(deletes); 365 } 366 } 367 } 368 369 // ========================================================================== 370 // Table Descriptors helpers 371 // ========================================================================== 372 private boolean hasGlobalReplicationScope(TableName tableName) throws IOException { 373 return hasGlobalReplicationScope(getTableDescriptor(tableName)); 374 } 375 376 private boolean hasGlobalReplicationScope(TableDescriptor htd) { 377 return htd != null ? htd.hasGlobalReplicationScope() : false; 378 } 379 380 private int getRegionReplication(TableDescriptor htd) { 381 return htd != null ? htd.getRegionReplication() : 1; 382 } 383 384 private TableDescriptor getTableDescriptor(TableName tableName) throws IOException { 385 return master.getTableDescriptors().get(tableName); 386 } 387 388 // ========================================================================== 389 // Region State 390 // ========================================================================== 391 392 /** 393 * Pull the region state from a catalog table {@link Result}. 394 * @return the region state, or null if unknown. 395 */ 396 public static State getRegionState(final Result r, RegionInfo regionInfo) { 397 Cell cell = 398 r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(regionInfo.getReplicaId())); 399 if (cell == null || cell.getValueLength() == 0) { 400 return null; 401 } 402 403 String state = 404 Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); 405 try { 406 return State.valueOf(state); 407 } catch (IllegalArgumentException e) { 408 LOG.warn( 409 "BAD value {} in hbase:meta info:state column for region {} , " 410 + "Consider using HBCK2 setRegionState ENCODED_REGION_NAME STATE", 411 state, regionInfo.getEncodedName()); 412 return null; 413 } 414 } 415 416 public static byte[] getStateColumn(int replicaId) { 417 return replicaId == 0 418 ? HConstants.STATE_QUALIFIER 419 : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER 420 + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); 421 } 422 423 private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException { 424 if (!METALOG.isDebugEnabled()) { 425 return; 426 } 427 // Logging each mutation in separate line makes it easier to see diff between them visually 428 // because of common starting indentation. 429 for (Mutation mutation : mutations) { 430 debugLogMutation(mutation); 431 } 432 } 433 434 private static void debugLogMutation(Mutation p) throws IOException { 435 METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON()); 436 } 437}