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:
Viraj Jasani 2019-10-11 12:58:50 +05:30 committed by Duo Zhang
parent f0b22120a0
commit 2f0b3ac91c
16 changed files with 197 additions and 196 deletions

View File

@ -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;
}
/**

View File

@ -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() {

View File

@ -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) {

View File

@ -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);
}
}

View File

@ -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

View File

@ -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);
}
}

View File

@ -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));
}
}
}

View File

@ -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

View File

@ -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

View File

@ -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);

View File

@ -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();
}
}
}

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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());

View File

@ -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));

View File

@ -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());
}
}