@@ -838,12 +838,13 @@ public CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName) {
838838 @ Override
839839 public CompletableFuture <List <RegionInfo >> getRegions (TableName tableName ) {
840840 if (tableName .equals (META_TABLE_NAME )) {
841- return connection .getLocator ().getRegionLocation (tableName , null , null , operationTimeoutNs )
842- .thenApply (loc -> Collections .singletonList (loc .getRegion ()));
841+ return connection .registry .getMetaRegionLocation ()
842+ .thenApply (locs -> Stream .of (locs .getRegionLocations ()).map (HRegionLocation ::getRegion )
843+ .collect (Collectors .toList ()));
843844 } else {
844845 return AsyncMetaTableAccessor .getTableHRegionLocations (metaTable , Optional .of (tableName ))
845- .thenApply (
846- locs -> locs .stream ().map (loc -> loc . getRegion () ).collect (Collectors .toList ()));
846+ .thenApply (
847+ locs -> locs .stream ().map (HRegionLocation :: getRegion ).collect (Collectors .toList ()));
847848 }
848849 }
849850
@@ -3418,21 +3419,24 @@ public CompletableFuture<Void> disableTableReplication(TableName tableName) {
34183419
34193420 private CompletableFuture <byte [][]> getTableSplits (TableName tableName ) {
34203421 CompletableFuture <byte [][]> future = new CompletableFuture <>();
3421- addListener (getRegions (tableName ), (regions , err2 ) -> {
3422- if (err2 != null ) {
3423- future .completeExceptionally (err2 );
3424- return ;
3425- }
3426- if (regions .size () == 1 ) {
3427- future .complete (null );
3428- } else {
3429- byte [][] splits = new byte [regions .size () - 1 ][];
3430- for (int i = 1 ; i < regions .size (); i ++) {
3431- splits [i - 1 ] = regions .get (i ).getStartKey ();
3422+ addListener (
3423+ getRegions (tableName ).thenApply (regions -> regions .stream ()
3424+ .filter (RegionReplicaUtil ::isDefaultReplica ).collect (Collectors .toList ())),
3425+ (regions , err2 ) -> {
3426+ if (err2 != null ) {
3427+ future .completeExceptionally (err2 );
3428+ return ;
34323429 }
3433- future .complete (splits );
3434- }
3435- });
3430+ if (regions .size () == 1 ) {
3431+ future .complete (null );
3432+ } else {
3433+ byte [][] splits = new byte [regions .size () - 1 ][];
3434+ for (int i = 1 ; i < regions .size (); i ++) {
3435+ splits [i - 1 ] = regions .get (i ).getStartKey ();
3436+ }
3437+ future .complete (splits );
3438+ }
3439+ });
34363440 return future ;
34373441 }
34383442
@@ -3661,13 +3665,15 @@ public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName n
36613665 if (err3 != null ) {
36623666 future .completeExceptionally (err3 );
36633667 } else {
3664- addListener (createTable (newTableDesc , splits ), (result , err4 ) -> {
3665- if (err4 != null ) {
3666- future .completeExceptionally (err4 );
3667- } else {
3668- future .complete (result );
3669- }
3670- });
3668+ addListener (
3669+ splits != null ? createTable (newTableDesc , splits ) : createTable (newTableDesc ),
3670+ (result , err4 ) -> {
3671+ if (err4 != null ) {
3672+ future .completeExceptionally (err4 );
3673+ } else {
3674+ future .complete (result );
3675+ }
3676+ });
36713677 }
36723678 });
36733679 } else {
0 commit comments