HBASE-24966 The methods in AsyncTableRegionLocator should not throw IOException directly (#2495)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
f879d45fb0
commit
8634428724
|
@ -17,7 +17,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -89,7 +88,6 @@ public interface AsyncTableRegionLocator {
|
||||||
* Find all the replicas for the region on which the given row is being served.
|
* Find all the replicas for the region on which the given row is being served.
|
||||||
* @param row Row to find.
|
* @param row Row to find.
|
||||||
* @return Locations for all the replicas of the row.
|
* @return Locations for all the replicas of the row.
|
||||||
* @throws IOException if a remote or network exception occurs
|
|
||||||
*/
|
*/
|
||||||
default CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row) {
|
default CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row) {
|
||||||
return getRegionLocations(row, false);
|
return getRegionLocations(row, false);
|
||||||
|
@ -100,7 +98,6 @@ public interface AsyncTableRegionLocator {
|
||||||
* @param row Row to find.
|
* @param row Row to find.
|
||||||
* @param reload true to reload information or false to use cached information
|
* @param reload true to reload information or false to use cached information
|
||||||
* @return Locations for all the replicas of the row.
|
* @return Locations for all the replicas of the row.
|
||||||
* @throws IOException if a remote or network exception occurs
|
|
||||||
*/
|
*/
|
||||||
CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row, boolean reload);
|
CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row, boolean reload);
|
||||||
|
|
||||||
|
@ -120,9 +117,8 @@ public interface AsyncTableRegionLocator {
|
||||||
* <p>
|
* <p>
|
||||||
* This is mainly useful for the MapReduce integration.
|
* This is mainly useful for the MapReduce integration.
|
||||||
* @return Array of region starting row keys
|
* @return Array of region starting row keys
|
||||||
* @throws IOException if a remote or network exception occurs
|
|
||||||
*/
|
*/
|
||||||
default CompletableFuture<List<byte[]>> getStartKeys() throws IOException {
|
default CompletableFuture<List<byte[]>> getStartKeys() {
|
||||||
return getStartEndKeys().thenApply(
|
return getStartEndKeys().thenApply(
|
||||||
startEndKeys -> startEndKeys.stream().map(Pair::getFirst).collect(Collectors.toList()));
|
startEndKeys -> startEndKeys.stream().map(Pair::getFirst).collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
|
@ -132,9 +128,8 @@ public interface AsyncTableRegionLocator {
|
||||||
* <p>
|
* <p>
|
||||||
* This is mainly useful for the MapReduce integration.
|
* This is mainly useful for the MapReduce integration.
|
||||||
* @return Array of region ending row keys
|
* @return Array of region ending row keys
|
||||||
* @throws IOException if a remote or network exception occurs
|
|
||||||
*/
|
*/
|
||||||
default CompletableFuture<List<byte[]>> getEndKeys() throws IOException {
|
default CompletableFuture<List<byte[]>> getEndKeys() {
|
||||||
return getStartEndKeys().thenApply(
|
return getStartEndKeys().thenApply(
|
||||||
startEndKeys -> startEndKeys.stream().map(Pair::getSecond).collect(Collectors.toList()));
|
startEndKeys -> startEndKeys.stream().map(Pair::getSecond).collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
|
@ -144,9 +139,8 @@ public interface AsyncTableRegionLocator {
|
||||||
* <p>
|
* <p>
|
||||||
* This is mainly useful for the MapReduce integration.
|
* This is mainly useful for the MapReduce integration.
|
||||||
* @return Pair of arrays of region starting and ending row keys
|
* @return Pair of arrays of region starting and ending row keys
|
||||||
* @throws IOException if a remote or network exception occurs
|
|
||||||
*/
|
*/
|
||||||
default CompletableFuture<List<Pair<byte[], byte[]>>> getStartEndKeys() throws IOException {
|
default CompletableFuture<List<Pair<byte[], byte[]>>> getStartEndKeys() {
|
||||||
return getAllRegionLocations().thenApply(
|
return getAllRegionLocations().thenApply(
|
||||||
locs -> locs.stream().filter(loc -> RegionReplicaUtil.isDefaultReplica(loc.getRegion()))
|
locs -> locs.stream().filter(loc -> RegionReplicaUtil.isDefaultReplica(loc.getRegion()))
|
||||||
.map(HRegionLocation::getRegion).map(r -> Pair.newPair(r.getStartKey(), r.getEndKey()))
|
.map(HRegionLocation::getRegion).map(r -> Pair.newPair(r.getStartKey(), r.getEndKey()))
|
||||||
|
|
Loading…
Reference in New Issue