HBASE-18624 Added support for clearing BlockCache based on tablename
Signed-off-by: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
2413a10e67
commit
d78d1ee672
|
@ -0,0 +1,54 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Public
|
||||
public final class CacheEvictionStats {
|
||||
|
||||
private final long evictedBlocks;
|
||||
private final long maxCacheSize;
|
||||
|
||||
CacheEvictionStats(CacheEvictionStatsBuilder builder) {
|
||||
this.evictedBlocks = builder.evictedBlocks;
|
||||
this.maxCacheSize = builder.maxCacheSize;
|
||||
}
|
||||
|
||||
public long getEvictedBlocks() {
|
||||
return evictedBlocks;
|
||||
}
|
||||
|
||||
public long getMaxCacheSize() {
|
||||
return maxCacheSize;
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public static CacheEvictionStatsBuilder builder() {
|
||||
return new CacheEvictionStatsBuilder();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "CacheEvictionStats{" +
|
||||
"evictedBlocks=" + evictedBlocks +
|
||||
", maxCacheSize=" + maxCacheSize +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public final class CacheEvictionStatsBuilder {
|
||||
long evictedBlocks = 0;
|
||||
long maxCacheSize = 0;
|
||||
|
||||
CacheEvictionStatsBuilder() {
|
||||
}
|
||||
|
||||
public CacheEvictionStatsBuilder withEvictedBlocks(long evictedBlocks) {
|
||||
this.evictedBlocks = evictedBlocks;
|
||||
return this;
|
||||
}
|
||||
|
||||
public CacheEvictionStatsBuilder withMaxCacheSize(long maxCacheSize) {
|
||||
this.maxCacheSize = maxCacheSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public CacheEvictionStatsBuilder append(CacheEvictionStats stats) {
|
||||
this.evictedBlocks += stats.getEvictedBlocks();
|
||||
this.maxCacheSize += stats.getMaxCacheSize();
|
||||
return this;
|
||||
}
|
||||
|
||||
public CacheEvictionStats build() {
|
||||
return new CacheEvictionStats(this);
|
||||
}
|
||||
}
|
|
@ -31,6 +31,7 @@ import java.util.regex.Pattern;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.ClusterStatus.Option;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -1047,6 +1048,18 @@ public interface Admin extends Abortable, Closeable {
|
|||
*/
|
||||
boolean isBalancerEnabled() throws IOException;
|
||||
|
||||
/**
|
||||
* Clear all the blocks corresponding to this table from BlockCache. For expert-admins.
|
||||
* Calling this API will drop all the cached blocks specific to a table from BlockCache.
|
||||
* This can significantly impact the query performance as the subsequent queries will
|
||||
* have to retrieve the blocks from underlying filesystem.
|
||||
*
|
||||
* @param tableName table to clear block cache
|
||||
* @return CacheEvictionStats related to the eviction
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CacheEvictionStats clearBlockCache(final TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Invoke region normalizer. Can NOT run for various reasons. Check logs.
|
||||
*
|
||||
|
|
|
@ -46,6 +46,8 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.ClusterStatus.Option;
|
||||
import org.apache.hadoop.hbase.CompoundConfiguration;
|
||||
|
@ -111,6 +113,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
|
||||
|
@ -1475,6 +1478,51 @@ public class HBaseAdmin implements Admin {
|
|||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public CacheEvictionStats clearBlockCache(final TableName tableName) throws IOException {
|
||||
checkTableExists(tableName);
|
||||
CacheEvictionStatsBuilder cacheEvictionStats = CacheEvictionStats.builder();
|
||||
List<Pair<RegionInfo, ServerName>> pairs =
|
||||
MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
|
||||
for (Pair<RegionInfo, ServerName> pair: pairs) {
|
||||
if (pair.getFirst().isOffline() || pair.getSecond() == null) {
|
||||
continue;
|
||||
}
|
||||
try {
|
||||
cacheEvictionStats = cacheEvictionStats.append(
|
||||
clearBlockCache(pair.getSecond(), pair.getFirst()));
|
||||
} catch (NotServingRegionException e) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Failed to clear block cache for " + pair.getFirst() + " on " +
|
||||
pair.getSecond() + ": " + StringUtils.stringifyException(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
return cacheEvictionStats.build();
|
||||
}
|
||||
|
||||
private CacheEvictionStats clearBlockCache(final ServerName sn, final RegionInfo hri)
|
||||
throws IOException {
|
||||
HBaseRpcController controller = rpcControllerFactory.newController();
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
ClearRegionBlockCacheRequest request =
|
||||
RequestConverter.buildClearRegionBlockCacheRequest(hri.getRegionName());
|
||||
try {
|
||||
return ProtobufUtil.toCacheEvictionStats(
|
||||
admin.clearRegionBlockCache(controller, request).getStats());
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Invoke region normalizer. Can NOT run for various reasons. Check logs.
|
||||
*
|
||||
* @return True if region normalizer ran, false otherwise.
|
||||
*/
|
||||
@Override
|
||||
public boolean normalize() throws IOException {
|
||||
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.NavigableSet;
|
|||
import java.util.function.Function;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellBuilderType;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
|
|
|
@ -41,6 +41,7 @@ import java.util.stream.Collectors;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellBuilderType;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
|
@ -3385,4 +3386,18 @@ public final class ProtobufUtil {
|
|||
.filter(snap -> pattern != null ? pattern.matcher(snap.getName()).matches() : true)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static CacheEvictionStats toCacheEvictionStats(HBaseProtos.CacheEvictionStats cacheEvictionStats) {
|
||||
return CacheEvictionStats.builder()
|
||||
.withEvictedBlocks(cacheEvictionStats.getEvictedBlocks())
|
||||
.withMaxCacheSize(cacheEvictionStats.getMaxCacheSize())
|
||||
.build();
|
||||
}
|
||||
|
||||
public static HBaseProtos.CacheEvictionStats toCacheEvictionStats(CacheEvictionStats cacheEvictionStats) {
|
||||
return HBaseProtos.CacheEvictionStats.newBuilder()
|
||||
.setEvictedBlocks(cacheEvictionStats.getEvictedBlocks())
|
||||
.setMaxCacheSize(cacheEvictionStats.getMaxCacheSize())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
|
|||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -60,6 +61,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
|
||||
|
@ -1422,6 +1424,17 @@ public final class RequestConverter {
|
|||
return IsBalancerEnabledRequest.newBuilder().build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer ClearRegionBlockCacheRequest
|
||||
*
|
||||
* @return a ClearRegionBlockCacheRequest
|
||||
*/
|
||||
public static ClearRegionBlockCacheRequest buildClearRegionBlockCacheRequest(final byte[]
|
||||
regionName) {
|
||||
RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
|
||||
return ClearRegionBlockCacheRequest.newBuilder().addAllRegion(Lists.newArrayList(region)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer GetClusterStatusRequest
|
||||
*
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.SingleResponse;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -435,4 +436,14 @@ public final class ResponseConverter {
|
|||
|
||||
return metricMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer ClearRegionBlockCacheResponse
|
||||
*
|
||||
* @return a ClearRegionBlockCacheResponse
|
||||
*/
|
||||
public static AdminProtos.ClearRegionBlockCacheResponse buildClearRegionBlockCacheResponse(final HBaseProtos.CacheEvictionStats
|
||||
cacheEvictionStats) {
|
||||
return AdminProtos.ClearRegionBlockCacheResponse.newBuilder().setStats(cacheEvictionStats).build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -248,6 +248,14 @@ message ClearCompactionQueuesRequest {
|
|||
message ClearCompactionQueuesResponse {
|
||||
}
|
||||
|
||||
message ClearRegionBlockCacheRequest {
|
||||
repeated RegionSpecifier region = 1;
|
||||
}
|
||||
|
||||
message ClearRegionBlockCacheResponse {
|
||||
required CacheEvictionStats stats = 1;
|
||||
}
|
||||
|
||||
message ExecuteProceduresRequest {
|
||||
repeated OpenRegionRequest open_region = 1;
|
||||
repeated CloseRegionRequest close_region = 2;
|
||||
|
@ -310,6 +318,9 @@ service AdminService {
|
|||
rpc ClearCompactionQueues(ClearCompactionQueuesRequest)
|
||||
returns(ClearCompactionQueuesResponse);
|
||||
|
||||
rpc ClearRegionBlockCache(ClearRegionBlockCacheRequest)
|
||||
returns(ClearRegionBlockCacheResponse);
|
||||
|
||||
/** Fetches the RegionServer's view of space quotas */
|
||||
rpc GetSpaceQuotaSnapshots(GetSpaceQuotaSnapshotsRequest)
|
||||
returns(GetSpaceQuotaSnapshotsResponse);
|
||||
|
|
|
@ -241,3 +241,9 @@ message RegionServerInfo {
|
|||
optional int32 infoPort = 1;
|
||||
optional VersionInfo version_info = 2;
|
||||
}
|
||||
|
||||
message CacheEvictionStats {
|
||||
optional int64 evicted_blocks = 1;
|
||||
optional int64 bytes_evicted = 2;
|
||||
optional int64 max_cache_size = 3;
|
||||
}
|
|
@ -59,6 +59,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
|
@ -99,6 +100,7 @@ import org.apache.hadoop.hbase.executor.ExecutorService;
|
|||
import org.apache.hadoop.hbase.executor.ExecutorType;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.http.InfoServer;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
|
@ -3269,7 +3271,7 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
/**
|
||||
* Protected utility method for safely obtaining an HRegion handle.
|
||||
* Protected Utility method for safely obtaining an HRegion handle.
|
||||
*
|
||||
* @param regionName
|
||||
* Name of online {@link HRegion} to return
|
||||
|
@ -3613,6 +3615,22 @@ public class HRegionServer extends HasThread implements
|
|||
configurationManager.notifyAllObservers(conf);
|
||||
}
|
||||
|
||||
public CacheEvictionStats clearRegionBlockCache(Region region) {
|
||||
BlockCache blockCache = this.getCacheConfig().getBlockCache();
|
||||
long evictedBlocks = 0;
|
||||
|
||||
for(Store store : region.getStores()) {
|
||||
for(StoreFile hFile : store.getStorefiles()) {
|
||||
evictedBlocks += blockCache.evictBlocksByHfileName(hFile.getPath().getName());
|
||||
}
|
||||
}
|
||||
|
||||
return CacheEvictionStats.builder()
|
||||
.withEvictedBlocks(evictedBlocks)
|
||||
.withMaxCacheSize(blockCache.getMaxSize())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getCompactionPressure() {
|
||||
double max = 0;
|
||||
|
|
|
@ -49,6 +49,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
|
@ -130,6 +132,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.cache.Cache;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
|
||||
|
@ -142,6 +145,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
|
@ -1361,19 +1366,24 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
@VisibleForTesting
|
||||
public HRegion getRegion(
|
||||
final RegionSpecifier regionSpecifier) throws IOException {
|
||||
ByteString value = regionSpecifier.getValue();
|
||||
RegionSpecifierType type = regionSpecifier.getType();
|
||||
switch (type) {
|
||||
case REGION_NAME:
|
||||
byte[] regionName = value.toByteArray();
|
||||
String encodedRegionName = RegionInfo.encodeRegionName(regionName);
|
||||
return regionServer.getRegionByEncodedName(regionName, encodedRegionName);
|
||||
case ENCODED_REGION_NAME:
|
||||
return regionServer.getRegionByEncodedName(value.toStringUtf8());
|
||||
default:
|
||||
throw new DoNotRetryIOException(
|
||||
"Unsupported region specifier type: " + type);
|
||||
return regionServer.getRegion(regionSpecifier.getValue().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* Find the List of HRegions based on a list of region specifiers
|
||||
*
|
||||
* @param regionSpecifiers the list of region specifiers
|
||||
* @return the corresponding list of regions
|
||||
* @throws IOException if any of the specifiers is not null,
|
||||
* but failed to find the region
|
||||
*/
|
||||
private List<HRegion> getRegions(
|
||||
final List<RegionSpecifier> regionSpecifiers) throws IOException {
|
||||
List<HRegion> regions = Lists.newArrayListWithCapacity(regionSpecifiers.size());
|
||||
for (RegionSpecifier regionSpecifier: regionSpecifiers) {
|
||||
regions.add(regionServer.getRegion(regionSpecifier.getValue().toByteArray()));
|
||||
}
|
||||
return regions;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -3442,15 +3452,33 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
ExecuteProceduresRequest request) throws ServiceException {
|
||||
ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
|
||||
if (request.getOpenRegionCount() > 0) {
|
||||
for (OpenRegionRequest req: request.getOpenRegionList()) {
|
||||
for (OpenRegionRequest req : request.getOpenRegionList()) {
|
||||
builder.addOpenRegion(openRegion(controller, req));
|
||||
}
|
||||
}
|
||||
if (request.getCloseRegionCount() > 0) {
|
||||
for (CloseRegionRequest req: request.getCloseRegionList()) {
|
||||
builder.addCloseRegion(closeRegion(controller, req));
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
if (request.getCloseRegionCount() > 0) {
|
||||
for (CloseRegionRequest req : request.getCloseRegionList()) {
|
||||
builder.addCloseRegion(closeRegion(controller, req));
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
|
||||
ClearRegionBlockCacheRequest request)
|
||||
throws ServiceException {
|
||||
CacheEvictionStatsBuilder stats = CacheEvictionStats.builder();
|
||||
try {
|
||||
List<HRegion> regions = getRegions(request.getRegionList());
|
||||
for (HRegion region : regions) {
|
||||
stats = stats.append(this.regionServer.clearRegionBlockCache(region));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
return ClearRegionBlockCacheResponse.newBuilder()
|
||||
.setStats(ProtobufUtil.toCacheEvictionStats(stats.build()))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,6 +74,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
|
@ -590,6 +592,13 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
|
||||
ClearRegionBlockCacheRequest request)
|
||||
throws ServiceException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HeapMemoryManager getHeapMemoryManager() {
|
||||
return null;
|
||||
|
|
|
@ -0,0 +1,125 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestClearRegionBlockCache {
|
||||
private static final TableName TABLE_NAME = TableName.valueOf("testClearRegionBlockCache");
|
||||
private static final byte[] FAMILY = Bytes.toBytes("family");
|
||||
private static final byte[][] SPLIT_KEY = new byte[][] { Bytes.toBytes("5") };
|
||||
private static final int NUM_MASTERS = 1;
|
||||
private static final int NUM_RS = 2;
|
||||
|
||||
private final HBaseTestingUtility HTU = new HBaseTestingUtility();
|
||||
|
||||
private Configuration CONF = HTU.getConfiguration();
|
||||
private Table table;
|
||||
private HRegionServer rs1, rs2;
|
||||
private MiniHBaseCluster cluster;
|
||||
|
||||
@Parameterized.Parameter public String cacheType;
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: {0}")
|
||||
public static Object[] data() {
|
||||
return new Object[] { "lru", "bucket" };
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
if (cacheType.equals("bucket")) {
|
||||
CONF.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
|
||||
CONF.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 30);
|
||||
}
|
||||
|
||||
cluster = HTU.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
rs1 = cluster.getRegionServer(0);
|
||||
rs2 = cluster.getRegionServer(1);
|
||||
|
||||
// Create table
|
||||
table = HTU.createTable(TABLE_NAME, FAMILY, SPLIT_KEY);
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
HTU.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClearBlockCache() throws Exception {
|
||||
HTU.loadNumericRows(table, FAMILY, 1, 10);
|
||||
HTU.flush(TABLE_NAME);
|
||||
|
||||
BlockCache blockCache1 = rs1.getCacheConfig().getBlockCache();
|
||||
BlockCache blockCache2 = rs2.getCacheConfig().getBlockCache();
|
||||
|
||||
long initialBlockCount1 = blockCache1.getBlockCount();
|
||||
long initialBlockCount2 = blockCache2.getBlockCount();
|
||||
|
||||
// scan will cause blocks to be added in BlockCache
|
||||
scanAllRegionsForRS(rs1);
|
||||
assertEquals(blockCache1.getBlockCount() - initialBlockCount1,
|
||||
HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
|
||||
clearRegionBlockCache(rs1);
|
||||
|
||||
scanAllRegionsForRS(rs2);
|
||||
assertEquals(blockCache2.getBlockCount() - initialBlockCount2,
|
||||
HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
|
||||
clearRegionBlockCache(rs2);
|
||||
|
||||
assertEquals(initialBlockCount1, blockCache1.getBlockCount());
|
||||
assertEquals(initialBlockCount2, blockCache2.getBlockCount());
|
||||
}
|
||||
|
||||
private void scanAllRegionsForRS(HRegionServer rs) throws IOException {
|
||||
for (Region region : rs.getRegions(TABLE_NAME)) {
|
||||
RegionScanner scanner = region.getScanner(new Scan());
|
||||
while (scanner.next(new ArrayList<Cell>()));
|
||||
}
|
||||
}
|
||||
|
||||
private void clearRegionBlockCache(HRegionServer rs) {
|
||||
for (Region region : rs.getRegions(TABLE_NAME)) {
|
||||
rs.clearRegionBlockCache(region);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -399,6 +399,12 @@ public class TestReplicator extends TestReplicationBase {
|
|||
throws ServiceException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
|
||||
ClearRegionBlockCacheRequest request) throws ServiceException {
|
||||
return delegate.clearRegionBlockCache(controller, request);
|
||||
}
|
||||
}
|
||||
|
||||
public class FailureInjectingReplicatorForTest extends ReplicatorForTest {
|
||||
|
|
|
@ -196,6 +196,12 @@ module Hbase
|
|||
@admin.isBalancerEnabled
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Requests clear block cache for table
|
||||
def clear_block_cache(table_name)
|
||||
@admin.clearBlockCache(org.apache.hadoop.hbase.TableName.valueOf(table_name)).toString
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Requests region normalization for all configured tables in the cluster
|
||||
# Returns true if normalizer ran successfully
|
||||
|
|
|
@ -360,6 +360,7 @@ Shell.load_command_group(
|
|||
clear_compaction_queues
|
||||
list_deadservers
|
||||
clear_deadservers
|
||||
clear_block_cache
|
||||
],
|
||||
# TODO: remove older hlog_roll command
|
||||
aliases: {
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
#
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
#
|
||||
|
||||
module Shell
|
||||
module Commands
|
||||
class ClearBlockCache < Command
|
||||
def help
|
||||
<<-EOF
|
||||
Clear all the blocks corresponding to this table from BlockCache. For expert-admins.
|
||||
Calling this API will drop all the cached blocks specific to a table from BlockCache.
|
||||
This can significantly impact the query performance as the subsequent queries will
|
||||
have to retrieve the blocks from underlying filesystem.
|
||||
For example:
|
||||
|
||||
hbase> clear_block_cache 'TABLENAME'
|
||||
EOF
|
||||
end
|
||||
|
||||
def command(table_name)
|
||||
formatter.row([admin.clear_block_cache(table_name)])
|
||||
end
|
||||
end
|
||||
end
|
||||
end
|
Loading…
Reference in New Issue