HBASE-23093 Avoid Optional Anti-Pattern where possible (#673)
Signed-off-by: Peter Somogyi <psomogyi@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
f0b22120a0
commit
2f0b3ac91c
|
@ -176,7 +176,7 @@ public class AsyncMetaTableAccessor {
|
|||
* {@link CompletableFuture}.
|
||||
*/
|
||||
public static CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable, Optional<TableName> tableName) {
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable, TableName tableName) {
|
||||
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
|
||||
addListener(getTableRegionsAndLocations(metaTable, tableName, true), (locations, err) -> {
|
||||
if (err != null) {
|
||||
|
@ -202,26 +202,28 @@ public class AsyncMetaTableAccessor {
|
|||
* {@link CompletableFuture}.
|
||||
*/
|
||||
private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable, final Optional<TableName> tableName,
|
||||
final boolean excludeOfflinedSplitParents) {
|
||||
final AsyncTable<AdvancedScanResultConsumer> metaTable,
|
||||
final TableName tableName, final boolean excludeOfflinedSplitParents) {
|
||||
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
|
||||
if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
|
||||
if (TableName.META_TABLE_NAME.equals(tableName)) {
|
||||
future.completeExceptionally(new IOException(
|
||||
"This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
|
||||
}
|
||||
|
||||
// Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
|
||||
CollectingVisitor<Pair<RegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
|
||||
private Optional<RegionLocations> current = null;
|
||||
CollectingVisitor<Pair<RegionInfo, ServerName>> visitor =
|
||||
new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
|
||||
private RegionLocations current = null;
|
||||
|
||||
@Override
|
||||
public boolean visit(Result r) throws IOException {
|
||||
current = getRegionLocations(r);
|
||||
if (!current.isPresent() || current.get().getRegionLocation().getRegion() == null) {
|
||||
Optional<RegionLocations> currentRegionLocations = getRegionLocations(r);
|
||||
current = currentRegionLocations.orElse(null);
|
||||
if (current == null || current.getRegionLocation().getRegion() == null) {
|
||||
LOG.warn("No serialized RegionInfo in " + r);
|
||||
return true;
|
||||
}
|
||||
RegionInfo hri = current.get().getRegionLocation().getRegion();
|
||||
RegionInfo hri = current.getRegionLocation().getRegion();
|
||||
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
|
||||
// Else call super and add this Result to the collection.
|
||||
return super.visit(r);
|
||||
|
@ -229,10 +231,10 @@ public class AsyncMetaTableAccessor {
|
|||
|
||||
@Override
|
||||
void add(Result r) {
|
||||
if (!current.isPresent()) {
|
||||
if (current == null) {
|
||||
return;
|
||||
}
|
||||
for (HRegionLocation loc : current.get().getRegionLocations()) {
|
||||
for (HRegionLocation loc : current.getRegionLocations()) {
|
||||
if (loc != null) {
|
||||
this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegion(), loc
|
||||
.getServerName()));
|
||||
|
@ -259,7 +261,7 @@ public class AsyncMetaTableAccessor {
|
|||
* @param visitor Visitor invoked against each row
|
||||
*/
|
||||
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
|
||||
Optional<TableName> tableName, QueryType type, final Visitor visitor) {
|
||||
TableName tableName, QueryType type, final Visitor visitor) {
|
||||
return scanMeta(metaTable, getTableStartRowForMeta(tableName, type),
|
||||
getTableStopRowForMeta(tableName, type), type, Integer.MAX_VALUE, visitor);
|
||||
}
|
||||
|
@ -274,15 +276,18 @@ public class AsyncMetaTableAccessor {
|
|||
* @param visitor Visitor invoked against each row
|
||||
*/
|
||||
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
|
||||
Optional<byte[]> startRow, Optional<byte[]> stopRow, QueryType type, int maxRows,
|
||||
final Visitor visitor) {
|
||||
byte[] startRow, byte[] stopRow, QueryType type, int maxRows, final Visitor visitor) {
|
||||
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
|
||||
Scan scan = getMetaScan(metaTable, rowUpperLimit);
|
||||
for (byte[] family : type.getFamilies()) {
|
||||
scan.addFamily(family);
|
||||
}
|
||||
startRow.ifPresent(scan::withStartRow);
|
||||
stopRow.ifPresent(scan::withStopRow);
|
||||
if (startRow != null) {
|
||||
scan.withStartRow(startRow);
|
||||
}
|
||||
if (stopRow != null) {
|
||||
scan.withStopRow(stopRow);
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Scanning META" + " starting at row=" + Bytes.toStringBinary(scan.getStartRow())
|
||||
|
@ -466,52 +471,56 @@ public class AsyncMetaTableAccessor {
|
|||
* @param tableName table we're working with
|
||||
* @return start row for scanning META according to query type
|
||||
*/
|
||||
private static Optional<byte[]> getTableStartRowForMeta(Optional<TableName> tableName,
|
||||
QueryType type) {
|
||||
return tableName.map((table) -> {
|
||||
switch (type) {
|
||||
case REGION:
|
||||
case REPLICATION:
|
||||
byte[] startRow = new byte[table.getName().length + 2];
|
||||
System.arraycopy(table.getName(), 0, startRow, 0, table.getName().length);
|
||||
startRow[startRow.length - 2] = HConstants.DELIMITER;
|
||||
startRow[startRow.length - 1] = HConstants.DELIMITER;
|
||||
return startRow;
|
||||
case ALL:
|
||||
case TABLE:
|
||||
default:
|
||||
return table.getName();
|
||||
private static byte[] getTableStartRowForMeta(TableName tableName, QueryType type) {
|
||||
if (tableName == null) {
|
||||
return null;
|
||||
}
|
||||
switch (type) {
|
||||
case REGION:
|
||||
case REPLICATION: {
|
||||
byte[] startRow = new byte[tableName.getName().length + 2];
|
||||
System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
|
||||
startRow[startRow.length - 2] = HConstants.DELIMITER;
|
||||
startRow[startRow.length - 1] = HConstants.DELIMITER;
|
||||
return startRow;
|
||||
}
|
||||
});
|
||||
case ALL:
|
||||
case TABLE:
|
||||
default: {
|
||||
return tableName.getName();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param tableName table we're working with
|
||||
* @return stop row for scanning META according to query type
|
||||
*/
|
||||
private static Optional<byte[]> getTableStopRowForMeta(Optional<TableName> tableName,
|
||||
QueryType type) {
|
||||
return tableName.map((table) -> {
|
||||
final byte[] stopRow;
|
||||
switch (type) {
|
||||
case REGION:
|
||||
case REPLICATION:
|
||||
stopRow = new byte[table.getName().length + 3];
|
||||
System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length);
|
||||
stopRow[stopRow.length - 3] = ' ';
|
||||
stopRow[stopRow.length - 2] = HConstants.DELIMITER;
|
||||
stopRow[stopRow.length - 1] = HConstants.DELIMITER;
|
||||
break;
|
||||
case ALL:
|
||||
case TABLE:
|
||||
default:
|
||||
stopRow = new byte[table.getName().length + 1];
|
||||
System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length);
|
||||
stopRow[stopRow.length - 1] = ' ';
|
||||
break;
|
||||
private static byte[] getTableStopRowForMeta(TableName tableName, QueryType type) {
|
||||
if (tableName == null) {
|
||||
return null;
|
||||
}
|
||||
final byte[] stopRow;
|
||||
switch (type) {
|
||||
case REGION:
|
||||
case REPLICATION: {
|
||||
stopRow = new byte[tableName.getName().length + 3];
|
||||
System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
|
||||
stopRow[stopRow.length - 3] = ' ';
|
||||
stopRow[stopRow.length - 2] = HConstants.DELIMITER;
|
||||
stopRow[stopRow.length - 1] = HConstants.DELIMITER;
|
||||
break;
|
||||
}
|
||||
return stopRow;
|
||||
});
|
||||
case ALL:
|
||||
case TABLE:
|
||||
default: {
|
||||
stopRow = new byte[tableName.getName().length + 1];
|
||||
System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
|
||||
stopRow[stopRow.length - 1] = ' ';
|
||||
break;
|
||||
}
|
||||
}
|
||||
return stopRow;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,7 +23,6 @@ import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
|
|||
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.removeRegionLocation;
|
||||
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.replaceRegionLocation;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
|
@ -110,7 +109,7 @@ class AsyncMetaRegionLocator {
|
|||
|
||||
void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
|
||||
AsyncRegionLocatorHelper.updateCachedLocationOnError(loc, exception, this::getCacheLocation,
|
||||
this::addLocationToCache, this::removeLocationFromCache, Optional.empty());
|
||||
this::addLocationToCache, this::removeLocationFromCache, null);
|
||||
}
|
||||
|
||||
void clearCache() {
|
||||
|
|
|
@ -145,7 +145,7 @@ class AsyncNonMetaRegionLocator {
|
|||
return allRequests.keySet().stream().filter(r -> !isPending(r)).findFirst();
|
||||
}
|
||||
|
||||
public void clearCompletedRequests(Optional<RegionLocations> locations) {
|
||||
public void clearCompletedRequests(RegionLocations locations) {
|
||||
for (Iterator<Map.Entry<LocateRequest, CompletableFuture<RegionLocations>>> iter =
|
||||
allRequests.entrySet().iterator(); iter.hasNext();) {
|
||||
Map.Entry<LocateRequest, CompletableFuture<RegionLocations>> entry = iter.next();
|
||||
|
@ -156,15 +156,14 @@ class AsyncNonMetaRegionLocator {
|
|||
}
|
||||
|
||||
private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations> future,
|
||||
Optional<RegionLocations> locations) {
|
||||
RegionLocations locations) {
|
||||
if (future.isDone()) {
|
||||
return true;
|
||||
}
|
||||
if (!locations.isPresent()) {
|
||||
if (locations == null) {
|
||||
return false;
|
||||
}
|
||||
RegionLocations locs = locations.get();
|
||||
HRegionLocation loc = ObjectUtils.firstNonNull(locs.getRegionLocations());
|
||||
HRegionLocation loc = ObjectUtils.firstNonNull(locations.getRegionLocations());
|
||||
// we should at least have one location available, otherwise the request should fail and
|
||||
// should not arrive here
|
||||
assert loc != null;
|
||||
|
@ -183,7 +182,7 @@ class AsyncNonMetaRegionLocator {
|
|||
completed = loc.getRegion().containsRow(req.row);
|
||||
}
|
||||
if (completed) {
|
||||
future.complete(locs);
|
||||
future.complete(locations);
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
|
@ -286,7 +285,7 @@ class AsyncNonMetaRegionLocator {
|
|||
RegionLocations addedLocs = addToCache(tableCache, locs);
|
||||
synchronized (tableCache) {
|
||||
tableCache.pendingRequests.remove(req);
|
||||
tableCache.clearCompletedRequests(Optional.of(addedLocs));
|
||||
tableCache.clearCompletedRequests(addedLocs);
|
||||
// Remove a complete locate request in a synchronized block, so the table cache must have
|
||||
// quota to send a candidate request.
|
||||
toSend = tableCache.getCandidate();
|
||||
|
@ -304,7 +303,7 @@ class AsyncNonMetaRegionLocator {
|
|||
if (future != null) {
|
||||
future.completeExceptionally(error);
|
||||
}
|
||||
tableCache.clearCompletedRequests(Optional.empty());
|
||||
tableCache.clearCompletedRequests(null);
|
||||
// Remove a complete locate request in a synchronized block, so the table cache must have
|
||||
// quota to send a candidate request.
|
||||
toSend = tableCache.getCandidate();
|
||||
|
@ -491,7 +490,7 @@ class AsyncNonMetaRegionLocator {
|
|||
}
|
||||
RegionLocations addedLocs = addToCache(tableCache, locs);
|
||||
synchronized (tableCache) {
|
||||
tableCache.clearCompletedRequests(Optional.of(addedLocs));
|
||||
tableCache.clearCompletedRequests(addedLocs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -604,8 +603,9 @@ class AsyncNonMetaRegionLocator {
|
|||
}
|
||||
|
||||
void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
|
||||
Optional<MetricsConnection> connectionMetrics = conn.getConnectionMetrics();
|
||||
AsyncRegionLocatorHelper.updateCachedLocationOnError(loc, exception, this::getCachedLocation,
|
||||
this::addLocationToCache, this::removeLocationFromCache, conn.getConnectionMetrics());
|
||||
this::addLocationToCache, this::removeLocationFromCache, connectionMetrics.orElse(null));
|
||||
}
|
||||
|
||||
void clearCache(TableName tableName) {
|
||||
|
|
|
@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findExcept
|
|||
import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import org.apache.commons.lang3.ObjectUtils;
|
||||
|
@ -53,7 +52,7 @@ final class AsyncRegionLocatorHelper {
|
|||
static void updateCachedLocationOnError(HRegionLocation loc, Throwable exception,
|
||||
Function<HRegionLocation, HRegionLocation> cachedLocationSupplier,
|
||||
Consumer<HRegionLocation> addToCache, Consumer<HRegionLocation> removeFromCache,
|
||||
Optional<MetricsConnection> metrics) {
|
||||
MetricsConnection metrics) {
|
||||
HRegionLocation oldLoc = cachedLocationSupplier.apply(loc);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Try updating {} , the old value is {}, error={}", loc, oldLoc,
|
||||
|
@ -81,7 +80,9 @@ final class AsyncRegionLocatorHelper {
|
|||
addToCache.accept(newLoc);
|
||||
} else {
|
||||
LOG.debug("Try removing {} from cache", loc);
|
||||
metrics.ifPresent(m -> m.incrCacheDroppingExceptions(exception));
|
||||
if (metrics != null) {
|
||||
metrics.incrCacheDroppingExceptions(exception);
|
||||
}
|
||||
removeFromCache.accept(loc);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
|
@ -60,7 +59,7 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
|
|||
.thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
|
||||
}
|
||||
return AsyncMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME),
|
||||
Optional.of(tableName));
|
||||
tableName);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -724,7 +724,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.complete(false);
|
||||
} else {
|
||||
addListener(
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)),
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName),
|
||||
(locations, error1) -> {
|
||||
if (error1 != null) {
|
||||
future.completeExceptionally(error1);
|
||||
|
@ -849,7 +849,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
.thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion)
|
||||
.collect(Collectors.toList()));
|
||||
} else {
|
||||
return AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName))
|
||||
return AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName)
|
||||
.thenApply(
|
||||
locs -> locs.stream().map(HRegionLocation::getRegion).collect(Collectors.toList()));
|
||||
}
|
||||
|
@ -1089,7 +1089,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
return future;
|
||||
} else {
|
||||
// For non-meta table, we fetch all locations by scanning hbase:meta table
|
||||
return AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName));
|
||||
return AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
|
@ -135,12 +134,15 @@ public class HttpProxyExample {
|
|||
ctx.fireChannelInactive();
|
||||
}
|
||||
|
||||
private void write(ChannelHandlerContext ctx, HttpResponseStatus status) {
|
||||
write(ctx, status, null);
|
||||
}
|
||||
|
||||
private void write(ChannelHandlerContext ctx, HttpResponseStatus status,
|
||||
Optional<String> content) {
|
||||
String content) {
|
||||
DefaultFullHttpResponse resp;
|
||||
if (content.isPresent()) {
|
||||
ByteBuf buf =
|
||||
ctx.alloc().buffer().writeBytes(Bytes.toBytes(content.get()));
|
||||
if (content != null) {
|
||||
ByteBuf buf = ctx.alloc().buffer().writeBytes(Bytes.toBytes(content));
|
||||
resp = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, status, buf);
|
||||
resp.headers().set(HttpHeaderNames.CONTENT_LENGTH, buf.readableBytes());
|
||||
} else {
|
||||
|
@ -171,9 +173,9 @@ public class HttpProxyExample {
|
|||
byte[] value =
|
||||
r.getValue(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier));
|
||||
if (value != null) {
|
||||
write(ctx, HttpResponseStatus.OK, Optional.of(Bytes.toStringBinary(value)));
|
||||
write(ctx, HttpResponseStatus.OK, Bytes.toStringBinary(value));
|
||||
} else {
|
||||
write(ctx, HttpResponseStatus.NOT_FOUND, Optional.empty());
|
||||
write(ctx, HttpResponseStatus.NOT_FOUND);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -190,7 +192,7 @@ public class HttpProxyExample {
|
|||
if (e != null) {
|
||||
exceptionCaught(ctx, e);
|
||||
} else {
|
||||
write(ctx, HttpResponseStatus.OK, Optional.empty());
|
||||
write(ctx, HttpResponseStatus.OK);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -205,7 +207,7 @@ public class HttpProxyExample {
|
|||
put(ctx, req);
|
||||
break;
|
||||
default:
|
||||
write(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED, Optional.empty());
|
||||
write(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -213,10 +215,10 @@ public class HttpProxyExample {
|
|||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
|
||||
if (cause instanceof IllegalArgumentException) {
|
||||
write(ctx, HttpResponseStatus.BAD_REQUEST, Optional.of(cause.getMessage()));
|
||||
write(ctx, HttpResponseStatus.BAD_REQUEST, cause.getMessage());
|
||||
} else {
|
||||
write(ctx, HttpResponseStatus.INTERNAL_SERVER_ERROR,
|
||||
Optional.of(Throwables.getStackTraceAsString(cause)));
|
||||
Throwables.getStackTraceAsString(cause));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -403,7 +403,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
/** Saved state from replaying prepare flush cache */
|
||||
private PrepareFlushResult prepareFlushResult = null;
|
||||
|
||||
private volatile Optional<ConfigurationManager> configurationManager;
|
||||
private volatile ConfigurationManager configurationManager;
|
||||
|
||||
// Used for testing.
|
||||
private volatile Long timeoutForWriteLock = null;
|
||||
|
@ -854,7 +854,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
LOG.debug("Instantiated " + this +"; "+ storeHotnessProtector.toString());
|
||||
}
|
||||
|
||||
configurationManager = Optional.empty();
|
||||
configurationManager = null;
|
||||
|
||||
// disable stats tracking system tables, but check the config for everything else
|
||||
this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
|
||||
|
@ -8735,7 +8735,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
*/
|
||||
@Override
|
||||
public void registerChildren(ConfigurationManager manager) {
|
||||
configurationManager = Optional.of(manager);
|
||||
configurationManager = manager;
|
||||
stores.values().forEach(manager::registerObserver);
|
||||
}
|
||||
|
||||
|
@ -8744,7 +8744,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
*/
|
||||
@Override
|
||||
public void deregisterChildren(ConfigurationManager manager) {
|
||||
stores.values().forEach(configurationManager.get()::deregisterObserver);
|
||||
stores.values().forEach(configurationManager::deregisterObserver);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalDouble;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
@ -61,11 +60,11 @@ class MetricsRegionServerWrapperImpl
|
|||
private final MetricsWALSource metricsWALSource;
|
||||
private final ByteBuffAllocator allocator;
|
||||
|
||||
private Optional<BlockCache> blockCache;
|
||||
private Optional<MobFileCache> mobFileCache;
|
||||
private Optional<CacheStats> cacheStats;
|
||||
private Optional<CacheStats> l1Stats = Optional.empty();
|
||||
private Optional<CacheStats> l2Stats = Optional.empty();
|
||||
private BlockCache blockCache;
|
||||
private MobFileCache mobFileCache;
|
||||
private CacheStats cacheStats;
|
||||
private CacheStats l1Stats = null;
|
||||
private CacheStats l2Stats = null;
|
||||
|
||||
private volatile long numStores = 0;
|
||||
private volatile long numWALFiles = 0;
|
||||
|
@ -152,14 +151,14 @@ class MetricsRegionServerWrapperImpl
|
|||
}
|
||||
|
||||
private void initBlockCache() {
|
||||
this.blockCache = this.regionServer.getBlockCache();
|
||||
this.cacheStats = this.blockCache.map(BlockCache::getStats);
|
||||
if (this.cacheStats.isPresent()) {
|
||||
if (this.cacheStats.get() instanceof CombinedBlockCache.CombinedCacheStats) {
|
||||
l1Stats = Optional
|
||||
.of(((CombinedBlockCache.CombinedCacheStats) this.cacheStats.get()).getLruCacheStats());
|
||||
l2Stats = Optional.of(((CombinedBlockCache.CombinedCacheStats) this.cacheStats.get())
|
||||
.getBucketCacheStats());
|
||||
this.blockCache = this.regionServer.getBlockCache().orElse(null);
|
||||
this.cacheStats = this.blockCache != null ? this.blockCache.getStats() : null;
|
||||
if (this.cacheStats != null) {
|
||||
if (this.cacheStats instanceof CombinedBlockCache.CombinedCacheStats) {
|
||||
l1Stats = ((CombinedBlockCache.CombinedCacheStats) this.cacheStats)
|
||||
.getLruCacheStats();
|
||||
l2Stats = ((CombinedBlockCache.CombinedCacheStats) this.cacheStats)
|
||||
.getBucketCacheStats();
|
||||
} else {
|
||||
l1Stats = this.cacheStats;
|
||||
}
|
||||
|
@ -170,7 +169,7 @@ class MetricsRegionServerWrapperImpl
|
|||
* Initializes the mob file cache.
|
||||
*/
|
||||
private void initMobFileCache() {
|
||||
this.mobFileCache = this.regionServer.getMobFileCache();
|
||||
this.mobFileCache = this.regionServer.getMobFileCache().orElse(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -275,7 +274,7 @@ class MetricsRegionServerWrapperImpl
|
|||
|
||||
@Override
|
||||
public long getBlockCacheCount() {
|
||||
return this.blockCache.map(BlockCache::getBlockCount).orElse(0L);
|
||||
return this.blockCache != null ? this.blockCache.getBlockCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -285,47 +284,47 @@ class MetricsRegionServerWrapperImpl
|
|||
|
||||
@Override
|
||||
public long getBlockCacheSize() {
|
||||
return this.blockCache.map(BlockCache::getCurrentSize).orElse(0L);
|
||||
return this.blockCache != null ? this.blockCache.getCurrentSize() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCacheFreeSize() {
|
||||
return this.blockCache.map(BlockCache::getFreeSize).orElse(0L);
|
||||
return this.blockCache != null ? this.blockCache.getFreeSize() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCacheHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCachePrimaryHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getPrimaryHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getPrimaryHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCacheMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCachePrimaryMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getPrimaryMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getPrimaryMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCacheEvictedCount() {
|
||||
return this.cacheStats.map(CacheStats::getEvictedCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getEvictedCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockCachePrimaryEvictedCount() {
|
||||
return this.cacheStats.map(CacheStats::getPrimaryEvictedCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getPrimaryEvictedCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getBlockCacheHitPercent() {
|
||||
double ratio = this.cacheStats.map(CacheStats::getHitRatio).orElse(0.0);
|
||||
double ratio = this.cacheStats != null ? this.cacheStats.getHitRatio() : 0.0;
|
||||
if (Double.isNaN(ratio)) {
|
||||
ratio = 0;
|
||||
}
|
||||
|
@ -334,7 +333,7 @@ class MetricsRegionServerWrapperImpl
|
|||
|
||||
@Override
|
||||
public double getBlockCacheHitCachingPercent() {
|
||||
double ratio = this.cacheStats.map(CacheStats::getHitCachingRatio).orElse(0.0);
|
||||
double ratio = this.cacheStats != null ? this.cacheStats.getHitCachingRatio() : 0.0;
|
||||
if (Double.isNaN(ratio)) {
|
||||
ratio = 0;
|
||||
}
|
||||
|
@ -343,47 +342,47 @@ class MetricsRegionServerWrapperImpl
|
|||
|
||||
@Override
|
||||
public long getBlockCacheFailedInsertions() {
|
||||
return this.cacheStats.map(CacheStats::getFailedInserts).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getFailedInserts() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getL1CacheHitCount() {
|
||||
return this.l1Stats.map(CacheStats::getHitCount).orElse(0L);
|
||||
return this.l1Stats != null ? this.l1Stats.getHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getL1CacheMissCount() {
|
||||
return this.l1Stats.map(CacheStats::getMissCount).orElse(0L);
|
||||
return this.l1Stats != null ? this.l1Stats.getMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getL1CacheHitRatio() {
|
||||
return this.l1Stats.map(CacheStats::getHitRatio).orElse(0.0);
|
||||
return this.l1Stats != null ? this.l1Stats.getHitRatio() : 0.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getL1CacheMissRatio() {
|
||||
return this.l1Stats.map(CacheStats::getMissRatio).orElse(0.0);
|
||||
return this.l1Stats != null ? this.l1Stats.getMissRatio() : 0.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getL2CacheHitCount() {
|
||||
return this.l2Stats.map(CacheStats::getHitCount).orElse(0L);
|
||||
return this.l2Stats != null ? this.l2Stats.getHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getL2CacheMissCount() {
|
||||
return this.l2Stats.map(CacheStats::getMissCount).orElse(0L);
|
||||
return this.l2Stats != null ? this.l2Stats.getMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getL2CacheHitRatio() {
|
||||
return this.l2Stats.map(CacheStats::getHitRatio).orElse(0.0);
|
||||
return this.l2Stats != null ? this.l2Stats.getHitRatio() : 0.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getL2CacheMissRatio() {
|
||||
return this.l2Stats.map(CacheStats::getMissRatio).orElse(0.0);
|
||||
return this.l2Stats != null ? this.l2Stats.getMissRatio() : 0.0;
|
||||
}
|
||||
|
||||
@Override public void forceRecompute() {
|
||||
|
@ -874,14 +873,14 @@ class MetricsRegionServerWrapperImpl
|
|||
mobFlushedCellsSize = tempMobFlushedCellsSize;
|
||||
mobScanCellsCount = tempMobScanCellsCount;
|
||||
mobScanCellsSize = tempMobScanCellsSize;
|
||||
mobFileCacheAccessCount = mobFileCache.map(MobFileCache::getAccessCount).orElse(0L);
|
||||
mobFileCacheMissCount = mobFileCache.map(MobFileCache::getMissCount).orElse(0L);
|
||||
mobFileCacheHitRatio = mobFileCache.map(MobFileCache::getHitRatio).orElse(0.0);
|
||||
mobFileCacheAccessCount = mobFileCache != null ? mobFileCache.getAccessCount() : 0L;
|
||||
mobFileCacheMissCount = mobFileCache != null ? mobFileCache.getMissCount() : 0L;
|
||||
mobFileCacheHitRatio = mobFileCache != null ? mobFileCache.getHitRatio() : 0.0;
|
||||
if (Double.isNaN(mobFileCacheHitRatio)) {
|
||||
mobFileCacheHitRatio = 0.0;
|
||||
}
|
||||
mobFileCacheEvictedCount = mobFileCache.map(MobFileCache::getEvictedFileCount).orElse(0L);
|
||||
mobFileCacheCount = mobFileCache.map(MobFileCache::getCacheSize).orElse(0);
|
||||
mobFileCacheEvictedCount = mobFileCache != null ? mobFileCache.getEvictedFileCount() : 0L;
|
||||
mobFileCacheCount = mobFileCache != null ? mobFileCache.getCacheSize() : 0;
|
||||
blockedRequestsCount = tempBlockedRequestsCount;
|
||||
} catch (Throwable e) {
|
||||
LOG.warn("Caught exception! Will suppress and retry.", e);
|
||||
|
@ -911,102 +910,102 @@ class MetricsRegionServerWrapperImpl
|
|||
|
||||
@Override
|
||||
public long getDataMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getDataMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getDataMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLeafIndexMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getLeafIndexMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getLeafIndexMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBloomChunkMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getBloomChunkMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getBloomChunkMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMetaMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getMetaMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getMetaMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRootIndexMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getRootIndexMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getRootIndexMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getIntermediateIndexMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getIntermediateIndexMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getIntermediateIndexMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileInfoMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getFileInfoMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getFileInfoMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getGeneralBloomMetaMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getGeneralBloomMetaMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getGeneralBloomMetaMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDeleteFamilyBloomMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getDeleteFamilyBloomMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getDeleteFamilyBloomMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTrailerMissCount() {
|
||||
return this.cacheStats.map(CacheStats::getTrailerMissCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getTrailerMissCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDataHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getDataHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getDataHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLeafIndexHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getLeafIndexHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getLeafIndexHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBloomChunkHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getBloomChunkHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getBloomChunkHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMetaHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getMetaHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getMetaHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRootIndexHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getRootIndexHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getRootIndexHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getIntermediateIndexHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getIntermediateIndexHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getIntermediateIndexHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileInfoHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getFileInfoHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getFileInfoHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getGeneralBloomMetaHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getGeneralBloomMetaHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getGeneralBloomMetaHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDeleteFamilyBloomHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getDeleteFamilyBloomHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getDeleteFamilyBloomHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTrailerHitCount() {
|
||||
return this.cacheStats.map(CacheStats::getTrailerHitCount).orElse(0L);
|
||||
return this.cacheStats != null ? this.cacheStats.getTrailerHitCount() : 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.InterruptedIOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
|
@ -351,11 +350,11 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
|
||||
// Used to instantiate a scanner for compaction in test
|
||||
@VisibleForTesting
|
||||
StoreScanner(ScanInfo scanInfo, OptionalInt maxVersions, ScanType scanType,
|
||||
StoreScanner(ScanInfo scanInfo, int maxVersions, ScanType scanType,
|
||||
List<? extends KeyValueScanner> scanners) throws IOException {
|
||||
// 0 is passed as readpoint because the test bypasses Store
|
||||
this(null, maxVersions.isPresent() ? new Scan().readVersions(maxVersions.getAsInt())
|
||||
: SCAN_FOR_COMPACTION, scanInfo, 0, 0L, false, scanType);
|
||||
this(null, maxVersions > 0 ? new Scan().readVersions(maxVersions)
|
||||
: SCAN_FOR_COMPACTION, scanInfo, 0, 0L, false, scanType);
|
||||
this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
|
||||
HConstants.OLDEST_TIMESTAMP, oldestUnexpiredTS, now, null, null, null);
|
||||
seekAllScanner(scanInfo, scanners);
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
|
|||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -54,7 +53,7 @@ class FSWALEntry extends Entry {
|
|||
private final transient boolean inMemstore;
|
||||
private final transient RegionInfo regionInfo;
|
||||
private final transient Set<byte[]> familyNames;
|
||||
private final transient Optional<ServerCall<?>> rpcCall;
|
||||
private final transient ServerCall<?> rpcCall;
|
||||
|
||||
FSWALEntry(final long txid, final WALKeyImpl key, final WALEdit edit, final RegionInfo regionInfo,
|
||||
final boolean inMemstore, ServerCall<?> rpcCall) {
|
||||
|
@ -69,7 +68,7 @@ class FSWALEntry extends Entry {
|
|||
} else {
|
||||
this.familyNames = Collections.<byte[]> emptySet();
|
||||
}
|
||||
this.rpcCall = Optional.ofNullable(rpcCall);
|
||||
this.rpcCall = rpcCall;
|
||||
if (rpcCall != null) {
|
||||
rpcCall.retainByWAL();
|
||||
}
|
||||
|
@ -135,6 +134,8 @@ class FSWALEntry extends Entry {
|
|||
}
|
||||
|
||||
void release() {
|
||||
rpcCall.ifPresent(ServerCall::releaseByWAL);
|
||||
if (rpcCall != null) {
|
||||
rpcCall.releaseByWAL();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -142,7 +142,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
|
|||
* operations (e.g. flush) and use the user HTD flag to decide whether or not replicate the
|
||||
* memstore.
|
||||
*/
|
||||
private boolean requiresReplication(Optional<TableDescriptor> tableDesc, Entry entry) {
|
||||
private boolean requiresReplication(Optional<TableDescriptor> tableDesc,
|
||||
Entry entry) {
|
||||
// empty edit does not need to be replicated
|
||||
if (entry.getEdit().isEmpty() || !tableDesc.isPresent()) {
|
||||
return false;
|
||||
|
|
|
@ -28,7 +28,6 @@ import static org.junit.Assert.fail;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
|
@ -83,7 +82,7 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get();
|
||||
int originalCount = regionLocations.size();
|
||||
|
||||
initSplitMergeSwitch();
|
||||
|
@ -114,7 +113,7 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get();
|
||||
int originalCount = regionLocations.size();
|
||||
|
||||
initSplitMergeSwitch();
|
||||
|
@ -158,8 +157,8 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
createTableWithDefaultConf(tableName, splitRows);
|
||||
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
List<HRegionLocation> regionLocations = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
RegionInfo regionA;
|
||||
RegionInfo regionB;
|
||||
|
||||
|
@ -169,16 +168,16 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
regionB = regionLocations.get(1).getRegion();
|
||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
regionLocations = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
assertEquals(2, regionLocations.size());
|
||||
// merge with encoded name
|
||||
regionA = regionLocations.get(0).getRegion();
|
||||
regionB = regionLocations.get(1).getRegion();
|
||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
regionLocations = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
assertEquals(1, regionLocations.size());
|
||||
}
|
||||
|
||||
|
@ -222,8 +221,8 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
createTableWithDefaultConf(tableName);
|
||||
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
List<HRegionLocation> regionLocations = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
assertEquals(1, regionLocations.size());
|
||||
|
||||
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||
|
@ -252,9 +251,8 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
int count = 0;
|
||||
for (int i = 0; i < 45; i++) {
|
||||
try {
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName))
|
||||
.get();
|
||||
regionLocations = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
count = regionLocations.size();
|
||||
if (count >= 2) {
|
||||
break;
|
||||
|
@ -267,4 +265,3 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
assertEquals(2, count);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -87,22 +87,20 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
|
||||
createTableWithDefaultConf(tableName);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
List<HRegionLocation> regionLocations = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
assertEquals("Table should have only 1 region", 1, regionLocations.size());
|
||||
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "_2");
|
||||
createTableWithDefaultConf(tableName2, new byte[][] { new byte[] { 42 } });
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
|
||||
regionLocations = AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName2).get();
|
||||
assertEquals("Table should have only 2 region", 2, regionLocations.size());
|
||||
|
||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName3);
|
||||
builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||
admin.createTable(builder.build(), Bytes.toBytes("a"), Bytes.toBytes("z"), 3).join();
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
|
||||
regionLocations = AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName3).get();
|
||||
assertEquals("Table should have only 3 region", 3, regionLocations.size());
|
||||
|
||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
|
||||
|
@ -119,8 +117,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
builder = TableDescriptorBuilder.newBuilder(tableName5);
|
||||
builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||
admin.createTable(builder.build(), new byte[] { 1 }, new byte[] { 127 }, 16).join();
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName5)).get();
|
||||
regionLocations = AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName5).get();
|
||||
assertEquals("Table should have 16 region", 16, regionLocations.size());
|
||||
}
|
||||
|
||||
|
@ -137,8 +134,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
|
||||
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
List<HRegionLocation> regions = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
Iterator<HRegionLocation> hris = regions.iterator();
|
||||
|
||||
assertEquals(
|
||||
|
@ -196,8 +193,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
||||
|
||||
regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
|
||||
regions = AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName2).get();
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
|
@ -249,8 +245,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
||||
|
||||
regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
|
||||
regions = AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, tableName3)
|
||||
.get();
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
|
|
|
@ -28,7 +28,6 @@ import static org.junit.Assert.fail;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
|
@ -284,8 +283,8 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
|
|||
createTableWithDefaultConf(tableName, splitKeys);
|
||||
|
||||
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
List<HRegionLocation> regions = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
|
@ -295,8 +294,8 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
|
|||
// Enable table, use retain assignment to assign regions.
|
||||
admin.enableTable(tableName).join();
|
||||
|
||||
List<HRegionLocation> regions2 =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
List<HRegionLocation> regions2 = AsyncMetaTableAccessor
|
||||
.getTableHRegionLocations(metaTable, tableName).get();
|
||||
// Check the assignment.
|
||||
assertEquals(regions.size(), regions2.size());
|
||||
assertTrue(regions2.containsAll(regions));
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -944,7 +943,7 @@ public class TestStoreScanner {
|
|||
200, /* timeToPurgeDeletes */
|
||||
CellComparator.getInstance(), false);
|
||||
try (StoreScanner scanner =
|
||||
new StoreScanner(scanInfo, OptionalInt.of(2), ScanType.COMPACT_DROP_DELETES, scanners)) {
|
||||
new StoreScanner(scanInfo, 2, ScanType.COMPACT_DROP_DELETES, scanners)) {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
results = new ArrayList<>();
|
||||
assertEquals(true, scanner.next(results));
|
||||
|
@ -972,8 +971,8 @@ public class TestStoreScanner {
|
|||
List<KeyValueScanner> scanners = scanFixture(kvs);
|
||||
ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
|
||||
HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.getInstance(), false);
|
||||
try (StoreScanner storeScanner = new StoreScanner(scanInfo, OptionalInt.empty(),
|
||||
ScanType.COMPACT_RETAIN_DELETES, scanners)) {
|
||||
try (StoreScanner storeScanner = new StoreScanner(scanInfo, -1,
|
||||
ScanType.COMPACT_RETAIN_DELETES, scanners)) {
|
||||
assertFalse(storeScanner.isScanUsePread());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue