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; 019 020import static org.apache.hadoop.hbase.util.FutureUtils.addListener; 021 022import java.io.IOException; 023import java.util.ArrayList; 024import java.util.Arrays; 025import java.util.Collections; 026import java.util.List; 027import java.util.Map; 028import java.util.NavigableMap; 029import java.util.Optional; 030import java.util.SortedMap; 031import java.util.concurrent.CompletableFuture; 032import java.util.regex.Matcher; 033import java.util.regex.Pattern; 034import java.util.stream.Collectors; 035import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor; 036import org.apache.hadoop.hbase.MetaTableAccessor.QueryType; 037import org.apache.hadoop.hbase.MetaTableAccessor.Visitor; 038import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer; 039import org.apache.hadoop.hbase.client.AsyncTable; 040import org.apache.hadoop.hbase.client.Consistency; 041import org.apache.hadoop.hbase.client.Get; 042import org.apache.hadoop.hbase.client.RegionInfo; 043import org.apache.hadoop.hbase.client.RegionReplicaUtil; 044import org.apache.hadoop.hbase.client.Result; 045import org.apache.hadoop.hbase.client.Scan; 046import org.apache.hadoop.hbase.client.Scan.ReadType; 047import org.apache.hadoop.hbase.client.TableState; 048import org.apache.hadoop.hbase.exceptions.DeserializationException; 049import org.apache.hadoop.hbase.util.Bytes; 050import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 051import org.apache.hadoop.hbase.util.Pair; 052import org.apache.yetus.audience.InterfaceAudience; 053import org.slf4j.Logger; 054import org.slf4j.LoggerFactory; 055 056/** 057 * The asynchronous meta table accessor. Used to read/write region and assignment information store 058 * in <code>hbase:meta</code>. 059 * @since 2.0.0 060 */ 061@InterfaceAudience.Private 062public class AsyncMetaTableAccessor { 063 064 private static final Logger LOG = LoggerFactory.getLogger(AsyncMetaTableAccessor.class); 065 066 067 /** The delimiter for meta columns for replicaIds > 0 */ 068 private static final char META_REPLICA_ID_DELIMITER = '_'; 069 070 /** A regex for parsing server columns from meta. See above javadoc for meta layout */ 071 private static final Pattern SERVER_COLUMN_PATTERN = Pattern 072 .compile("^server(_[0-9a-fA-F]{4})?$"); 073 074 public static CompletableFuture<Boolean> tableExists(AsyncTable<?> metaTable, 075 TableName tableName) { 076 return getTableState(metaTable, tableName).thenApply(Optional::isPresent); 077 } 078 079 public static CompletableFuture<Optional<TableState>> getTableState(AsyncTable<?> metaTable, 080 TableName tableName) { 081 CompletableFuture<Optional<TableState>> future = new CompletableFuture<>(); 082 Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn()); 083 long time = EnvironmentEdgeManager.currentTime(); 084 try { 085 get.setTimeRange(0, time); 086 addListener(metaTable.get(get), (result, error) -> { 087 if (error != null) { 088 future.completeExceptionally(error); 089 return; 090 } 091 try { 092 future.complete(getTableState(result)); 093 } catch (IOException e) { 094 future.completeExceptionally(e); 095 } 096 }); 097 } catch (IOException ioe) { 098 future.completeExceptionally(ioe); 099 } 100 return future; 101 } 102 103 /** 104 * Returns the HRegionLocation from meta for the given region 105 * @param metaTable 106 * @param regionName region we're looking for 107 * @return HRegionLocation for the given region 108 */ 109 public static CompletableFuture<Optional<HRegionLocation>> getRegionLocation( 110 AsyncTable<?> metaTable, byte[] regionName) { 111 CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>(); 112 try { 113 RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName); 114 addListener(metaTable.get(new Get(MetaTableAccessor.getMetaKeyForRegion(parsedRegionInfo)) 115 .addFamily(HConstants.CATALOG_FAMILY)), (r, err) -> { 116 if (err != null) { 117 future.completeExceptionally(err); 118 return; 119 } 120 future.complete(getRegionLocations(r) 121 .map(locations -> locations.getRegionLocation(parsedRegionInfo.getReplicaId()))); 122 }); 123 } catch (IOException parseEx) { 124 LOG.warn("Failed to parse the passed region name: " + Bytes.toStringBinary(regionName)); 125 future.completeExceptionally(parseEx); 126 } 127 return future; 128 } 129 130 /** 131 * Returns the HRegionLocation from meta for the given encoded region name 132 * @param metaTable 133 * @param encodedRegionName region we're looking for 134 * @return HRegionLocation for the given region 135 */ 136 public static CompletableFuture<Optional<HRegionLocation>> getRegionLocationWithEncodedName( 137 AsyncTable<?> metaTable, byte[] encodedRegionName) { 138 CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>(); 139 addListener( 140 metaTable 141 .scanAll(new Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY)), 142 (results, err) -> { 143 if (err != null) { 144 future.completeExceptionally(err); 145 return; 146 } 147 String encodedRegionNameStr = Bytes.toString(encodedRegionName); 148 results.stream().filter(result -> !result.isEmpty()) 149 .filter(result -> MetaTableAccessor.getRegionInfo(result) != null).forEach(result -> { 150 getRegionLocations(result).ifPresent(locations -> { 151 for (HRegionLocation location : locations.getRegionLocations()) { 152 if (location != null && 153 encodedRegionNameStr.equals(location.getRegion().getEncodedName())) { 154 future.complete(Optional.of(location)); 155 return; 156 } 157 } 158 }); 159 }); 160 future.complete(Optional.empty()); 161 }); 162 return future; 163 } 164 165 private static Optional<TableState> getTableState(Result r) throws IOException { 166 Cell cell = r.getColumnLatestCell(getTableFamily(), getStateColumn()); 167 if (cell == null) return Optional.empty(); 168 try { 169 return Optional.of(TableState.parseFrom( 170 TableName.valueOf(r.getRow()), 171 Arrays.copyOfRange(cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset() 172 + cell.getValueLength()))); 173 } catch (DeserializationException e) { 174 throw new IOException("Failed to parse table state from result: " + r, e); 175 } 176 } 177 178 /** 179 * Used to get all region locations for the specific table. 180 * @param metaTable 181 * @param tableName table we're looking for, can be null for getting all regions 182 * @return the list of region locations. The return value will be wrapped by a 183 * {@link CompletableFuture}. 184 */ 185 public static CompletableFuture<List<HRegionLocation>> getTableHRegionLocations( 186 AsyncTable<AdvancedScanResultConsumer> metaTable, Optional<TableName> tableName) { 187 CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>(); 188 addListener(getTableRegionsAndLocations(metaTable, tableName, true), (locations, err) -> { 189 if (err != null) { 190 future.completeExceptionally(err); 191 } else if (locations == null || locations.isEmpty()) { 192 future.complete(Collections.emptyList()); 193 } else { 194 List<HRegionLocation> regionLocations = 195 locations.stream().map(loc -> new HRegionLocation(loc.getFirst(), loc.getSecond())) 196 .collect(Collectors.toList()); 197 future.complete(regionLocations); 198 } 199 }); 200 return future; 201 } 202 203 /** 204 * Used to get table regions' info and server. 205 * @param metaTable 206 * @param tableName table we're looking for, can be null for getting all regions 207 * @param excludeOfflinedSplitParents don't return split parents 208 * @return the list of regioninfos and server. The return value will be wrapped by a 209 * {@link CompletableFuture}. 210 */ 211 private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations( 212 AsyncTable<AdvancedScanResultConsumer> metaTable, final Optional<TableName> tableName, 213 final boolean excludeOfflinedSplitParents) { 214 CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>(); 215 if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) { 216 future.completeExceptionally(new IOException( 217 "This method can't be used to locate meta regions;" + " use MetaTableLocator instead")); 218 } 219 220 // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress 221 CollectingVisitor<Pair<RegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<RegionInfo, ServerName>>() { 222 private Optional<RegionLocations> current = null; 223 224 @Override 225 public boolean visit(Result r) throws IOException { 226 current = getRegionLocations(r); 227 if (!current.isPresent() || current.get().getRegionLocation().getRegion() == null) { 228 LOG.warn("No serialized RegionInfo in " + r); 229 return true; 230 } 231 RegionInfo hri = current.get().getRegionLocation().getRegion(); 232 if (excludeOfflinedSplitParents && hri.isSplitParent()) return true; 233 // Else call super and add this Result to the collection. 234 return super.visit(r); 235 } 236 237 @Override 238 void add(Result r) { 239 if (!current.isPresent()) { 240 return; 241 } 242 for (HRegionLocation loc : current.get().getRegionLocations()) { 243 if (loc != null) { 244 this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegion(), loc 245 .getServerName())); 246 } 247 } 248 } 249 }; 250 251 addListener(scanMeta(metaTable, tableName, QueryType.REGION, visitor), (v, error) -> { 252 if (error != null) { 253 future.completeExceptionally(error); 254 return; 255 } 256 future.complete(visitor.getResults()); 257 }); 258 return future; 259 } 260 261 /** 262 * Performs a scan of META table for given table. 263 * @param metaTable 264 * @param tableName table withing we scan 265 * @param type scanned part of meta 266 * @param visitor Visitor invoked against each row 267 */ 268 private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable, 269 Optional<TableName> tableName, QueryType type, final Visitor visitor) { 270 return scanMeta(metaTable, getTableStartRowForMeta(tableName, type), 271 getTableStopRowForMeta(tableName, type), type, Integer.MAX_VALUE, visitor); 272 } 273 274 /** 275 * Performs a scan of META table for given table. 276 * @param metaTable 277 * @param startRow Where to start the scan 278 * @param stopRow Where to stop the scan 279 * @param type scanned part of meta 280 * @param maxRows maximum rows to return 281 * @param visitor Visitor invoked against each row 282 */ 283 private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable, 284 Optional<byte[]> startRow, Optional<byte[]> stopRow, QueryType type, int maxRows, 285 final Visitor visitor) { 286 int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE; 287 Scan scan = getMetaScan(metaTable, rowUpperLimit); 288 for (byte[] family : type.getFamilies()) { 289 scan.addFamily(family); 290 } 291 startRow.ifPresent(scan::withStartRow); 292 stopRow.ifPresent(scan::withStopRow); 293 294 if (LOG.isDebugEnabled()) { 295 LOG.debug("Scanning META" + " starting at row=" + Bytes.toStringBinary(scan.getStartRow()) 296 + " stopping at row=" + Bytes.toStringBinary(scan.getStopRow()) + " for max=" 297 + rowUpperLimit + " with caching=" + scan.getCaching()); 298 } 299 300 CompletableFuture<Void> future = new CompletableFuture<Void>(); 301 metaTable.scan(scan, new MetaTableScanResultConsumer(rowUpperLimit, visitor, future)); 302 return future; 303 } 304 305 private static final class MetaTableScanResultConsumer implements AdvancedScanResultConsumer { 306 307 private int currentRowCount; 308 309 private final int rowUpperLimit; 310 311 private final Visitor visitor; 312 313 private final CompletableFuture<Void> future; 314 315 MetaTableScanResultConsumer(int rowUpperLimit, Visitor visitor, 316 CompletableFuture<Void> future) { 317 this.rowUpperLimit = rowUpperLimit; 318 this.visitor = visitor; 319 this.future = future; 320 this.currentRowCount = 0; 321 } 322 323 @Override 324 public void onError(Throwable error) { 325 future.completeExceptionally(error); 326 } 327 328 @Override 329 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NONNULL_PARAM_VIOLATION", 330 justification = "https://github.com/findbugsproject/findbugs/issues/79") 331 public void onComplete() { 332 future.complete(null); 333 } 334 335 @Override 336 public void onNext(Result[] results, ScanController controller) { 337 boolean terminateScan = false; 338 for (Result result : results) { 339 try { 340 if (!visitor.visit(result)) { 341 terminateScan = true; 342 break; 343 } 344 } catch (Exception e) { 345 future.completeExceptionally(e); 346 terminateScan = true; 347 break; 348 } 349 if (++currentRowCount >= rowUpperLimit) { 350 terminateScan = true; 351 break; 352 } 353 } 354 if (terminateScan) { 355 controller.terminate(); 356 } 357 } 358 } 359 360 private static Scan getMetaScan(AsyncTable<?> metaTable, int rowUpperLimit) { 361 Scan scan = new Scan(); 362 int scannerCaching = metaTable.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING, 363 HConstants.DEFAULT_HBASE_META_SCANNER_CACHING); 364 if (metaTable.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS, 365 HConstants.DEFAULT_USE_META_REPLICAS)) { 366 scan.setConsistency(Consistency.TIMELINE); 367 } 368 if (rowUpperLimit <= scannerCaching) { 369 scan.setLimit(rowUpperLimit); 370 } 371 int rows = Math.min(rowUpperLimit, scannerCaching); 372 scan.setCaching(rows); 373 return scan; 374 } 375 376 /** 377 * Returns an HRegionLocationList extracted from the result. 378 * @return an HRegionLocationList containing all locations for the region range or null if we 379 * can't deserialize the result. 380 */ 381 private static Optional<RegionLocations> getRegionLocations(final Result r) { 382 if (r == null) return Optional.empty(); 383 Optional<RegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn()); 384 if (!regionInfo.isPresent()) return Optional.empty(); 385 386 List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1); 387 NavigableMap<byte[], NavigableMap<byte[], byte[]>> familyMap = r.getNoVersionMap(); 388 389 locations.add(getRegionLocation(r, regionInfo.get(), 0)); 390 391 NavigableMap<byte[], byte[]> infoMap = familyMap.get(getCatalogFamily()); 392 if (infoMap == null) return Optional.of(new RegionLocations(locations)); 393 394 // iterate until all serverName columns are seen 395 int replicaId = 0; 396 byte[] serverColumn = getServerColumn(replicaId); 397 SortedMap<byte[], byte[]> serverMap = null; 398 serverMap = infoMap.tailMap(serverColumn, false); 399 400 if (serverMap.isEmpty()) return Optional.of(new RegionLocations(locations)); 401 402 for (Map.Entry<byte[], byte[]> entry : serverMap.entrySet()) { 403 replicaId = parseReplicaIdFromServerColumn(entry.getKey()); 404 if (replicaId < 0) { 405 break; 406 } 407 HRegionLocation location = getRegionLocation(r, regionInfo.get(), replicaId); 408 // In case the region replica is newly created, it's location might be null. We usually do not 409 // have HRL's in RegionLocations object with null ServerName. They are handled as null HRLs. 410 if (location == null || location.getServerName() == null) { 411 locations.add(null); 412 } else { 413 locations.add(location); 414 } 415 } 416 417 return Optional.of(new RegionLocations(locations)); 418 } 419 420 /** 421 * Returns the HRegionLocation parsed from the given meta row Result 422 * for the given regionInfo and replicaId. The regionInfo can be the default region info 423 * for the replica. 424 * @param r the meta row result 425 * @param regionInfo RegionInfo for default replica 426 * @param replicaId the replicaId for the HRegionLocation 427 * @return HRegionLocation parsed from the given meta row Result for the given replicaId 428 */ 429 private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo, 430 final int replicaId) { 431 Optional<ServerName> serverName = getServerName(r, replicaId); 432 long seqNum = getSeqNumDuringOpen(r, replicaId); 433 RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId); 434 return new HRegionLocation(replicaInfo, serverName.orElse(null), seqNum); 435 } 436 437 /** 438 * Returns a {@link ServerName} from catalog table {@link Result}. 439 * @param r Result to pull from 440 * @return A ServerName instance. 441 */ 442 private static Optional<ServerName> getServerName(final Result r, final int replicaId) { 443 byte[] serverColumn = getServerColumn(replicaId); 444 Cell cell = r.getColumnLatestCell(getCatalogFamily(), serverColumn); 445 if (cell == null || cell.getValueLength() == 0) return Optional.empty(); 446 String hostAndPort = Bytes.toString(cell.getValueArray(), cell.getValueOffset(), 447 cell.getValueLength()); 448 byte[] startcodeColumn = getStartCodeColumn(replicaId); 449 cell = r.getColumnLatestCell(getCatalogFamily(), startcodeColumn); 450 if (cell == null || cell.getValueLength() == 0) return Optional.empty(); 451 try { 452 return Optional.of(ServerName.valueOf(hostAndPort, 453 Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()))); 454 } catch (IllegalArgumentException e) { 455 LOG.error("Ignoring invalid region for server " + hostAndPort + "; cell=" + cell, e); 456 return Optional.empty(); 457 } 458 } 459 460 /** 461 * The latest seqnum that the server writing to meta observed when opening the region. 462 * E.g. the seqNum when the result of {@link #getServerName(Result, int)} was written. 463 * @param r Result to pull the seqNum from 464 * @return SeqNum, or HConstants.NO_SEQNUM if there's no value written. 465 */ 466 private static long getSeqNumDuringOpen(final Result r, final int replicaId) { 467 Cell cell = r.getColumnLatestCell(getCatalogFamily(), getSeqNumColumn(replicaId)); 468 if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM; 469 return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); 470 } 471 472 /** 473 * @param tableName table we're working with 474 * @return start row for scanning META according to query type 475 */ 476 private static Optional<byte[]> getTableStartRowForMeta(Optional<TableName> tableName, 477 QueryType type) { 478 return tableName.map((table) -> { 479 switch (type) { 480 case REGION: 481 byte[] startRow = new byte[table.getName().length + 2]; 482 System.arraycopy(table.getName(), 0, startRow, 0, table.getName().length); 483 startRow[startRow.length - 2] = HConstants.DELIMITER; 484 startRow[startRow.length - 1] = HConstants.DELIMITER; 485 return startRow; 486 case ALL: 487 case TABLE: 488 default: 489 return table.getName(); 490 } 491 }); 492 } 493 494 /** 495 * @param tableName table we're working with 496 * @return stop row for scanning META according to query type 497 */ 498 private static Optional<byte[]> getTableStopRowForMeta(Optional<TableName> tableName, 499 QueryType type) { 500 return tableName.map((table) -> { 501 final byte[] stopRow; 502 switch (type) { 503 case REGION: 504 stopRow = new byte[table.getName().length + 3]; 505 System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length); 506 stopRow[stopRow.length - 3] = ' '; 507 stopRow[stopRow.length - 2] = HConstants.DELIMITER; 508 stopRow[stopRow.length - 1] = HConstants.DELIMITER; 509 break; 510 case ALL: 511 case TABLE: 512 default: 513 stopRow = new byte[table.getName().length + 1]; 514 System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length); 515 stopRow[stopRow.length - 1] = ' '; 516 break; 517 } 518 return stopRow; 519 }); 520 } 521 522 /** 523 * Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and 524 * <code>qualifier</code> of the catalog table result. 525 * @param r a Result object from the catalog table scan 526 * @param qualifier Column family qualifier 527 * @return An RegionInfo instance. 528 */ 529 private static Optional<RegionInfo> getHRegionInfo(final Result r, byte[] qualifier) { 530 Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier); 531 if (cell == null) return Optional.empty(); 532 return Optional.ofNullable(RegionInfo.parseFromOrNull(cell.getValueArray(), 533 cell.getValueOffset(), cell.getValueLength())); 534 } 535 536 /** 537 * Returns the column family used for meta columns. 538 * @return HConstants.CATALOG_FAMILY. 539 */ 540 private static byte[] getCatalogFamily() { 541 return HConstants.CATALOG_FAMILY; 542 } 543 544 /** 545 * Returns the column family used for table columns. 546 * @return HConstants.TABLE_FAMILY. 547 */ 548 private static byte[] getTableFamily() { 549 return HConstants.TABLE_FAMILY; 550 } 551 552 /** 553 * Returns the column qualifier for serialized region info 554 * @return HConstants.REGIONINFO_QUALIFIER 555 */ 556 private static byte[] getRegionInfoColumn() { 557 return HConstants.REGIONINFO_QUALIFIER; 558 } 559 560 /** 561 * Returns the column qualifier for serialized table state 562 * @return HConstants.TABLE_STATE_QUALIFIER 563 */ 564 private static byte[] getStateColumn() { 565 return HConstants.TABLE_STATE_QUALIFIER; 566 } 567 568 /** 569 * Returns the column qualifier for server column for replicaId 570 * @param replicaId the replicaId of the region 571 * @return a byte[] for server column qualifier 572 */ 573 private static byte[] getServerColumn(int replicaId) { 574 return replicaId == 0 575 ? HConstants.SERVER_QUALIFIER 576 : Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER 577 + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); 578 } 579 580 /** 581 * Returns the column qualifier for server start code column for replicaId 582 * @param replicaId the replicaId of the region 583 * @return a byte[] for server start code column qualifier 584 */ 585 private static byte[] getStartCodeColumn(int replicaId) { 586 return replicaId == 0 587 ? HConstants.STARTCODE_QUALIFIER 588 : Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER 589 + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); 590 } 591 592 /** 593 * Returns the column qualifier for seqNum column for replicaId 594 * @param replicaId the replicaId of the region 595 * @return a byte[] for seqNum column qualifier 596 */ 597 private static byte[] getSeqNumColumn(int replicaId) { 598 return replicaId == 0 599 ? HConstants.SEQNUM_QUALIFIER 600 : Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER 601 + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); 602 } 603 604 /** 605 * Parses the replicaId from the server column qualifier. See top of the class javadoc 606 * for the actual meta layout 607 * @param serverColumn the column qualifier 608 * @return an int for the replicaId 609 */ 610 private static int parseReplicaIdFromServerColumn(byte[] serverColumn) { 611 String serverStr = Bytes.toString(serverColumn); 612 613 Matcher matcher = SERVER_COLUMN_PATTERN.matcher(serverStr); 614 if (matcher.matches() && matcher.groupCount() > 0) { 615 String group = matcher.group(1); 616 if (group != null && group.length() > 0) { 617 return Integer.parseInt(group.substring(1), 16); 618 } else { 619 return 0; 620 } 621 } 622 return -1; 623 } 624}