diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index c0e2994c3c5e..3e1833635e29 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -2651,4 +2651,13 @@ List getLogEntries(Set serverNames, String logType, Server * Get the list of cached files */ List getCachedFilesList(ServerName serverName) throws IOException; + + /** + * Clean Cache by evicting the blocks of files belonging to regions that are no longer served by + * the RegionServer. + * @param serverName ServerName + * @return A map of filename and number of blocks evicted. + * @throws IOException if a remote or network exception occurs + */ + Map uncacheStaleBlocks(ServerName serverName) throws IOException; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java index c13dfc33e3d2..a04dda5b1c81 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java @@ -1136,4 +1136,9 @@ public void flushMasterStore() throws IOException { public List getCachedFilesList(ServerName serverName) throws IOException { return get(admin.getCachedFilesList(serverName)); } + + @Override + public Map uncacheStaleBlocks(ServerName serverName) throws IOException { + return get(admin.uncacheStaleBlocks(serverName)); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index bdb0228d9687..a9a286c3421d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -1861,4 +1861,12 @@ CompletableFuture> getLogEntries(Set serverNames, Str * Get the list of cached files */ CompletableFuture> getCachedFilesList(ServerName serverName); + + /** + * Clean Cache by evicting the blocks of files belonging to regions that are no longer served by + * the RegionServer. + * @param serverName ServerName + * @return A map of filename and number of blocks evicted. + */ + CompletableFuture> uncacheStaleBlocks(ServerName serverName); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 69f353600036..481f3b2bb464 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -1005,4 +1005,9 @@ public CompletableFuture flushMasterStore() { public CompletableFuture> getCachedFilesList(ServerName serverName) { return wrap(rawAdmin.getCachedFilesList(serverName)); } + + @Override + public CompletableFuture> uncacheStaleBlocks(ServerName serverName) { + return wrap(rawAdmin.uncacheStaleBlocks(serverName)); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 103a64e520a1..33ed2f7441b0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -144,6 +144,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; @@ -4537,4 +4539,15 @@ List> adminCall(controller, stub, request.build(), resp -> resp.getCachedFilesList())) .serverName(serverName).call(); } + + @Override + public CompletableFuture> uncacheStaleBlocks(ServerName serverName) { + UncacheStaleBlocksRequest.Builder request = UncacheStaleBlocksRequest.newBuilder(); + return this.> newAdminCaller() + .action((controller, stub) -> this.> adminCall(controller, stub, request.build(), + (s, c, req, done) -> s.uncacheStaleBlocks(c, req, done), + resp -> resp.getUncachedFilesMap())) + .serverName(serverName).call(); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index e50b54e8eb02..ca58ca50b42b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -166,6 +166,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; @@ -1835,6 +1837,22 @@ public static List getCachedFilesList(final RpcController controller, return new ArrayList<>(response.getCachedFilesList()); } + /** + * Clean Cache by evicting the blocks of files belonging to regions that are no longer served by + * the RegionServer. + */ + public static Map uncacheStaleBlocks(final RpcController controller, + final AdminService.BlockingInterface admin) throws IOException { + UncacheStaleBlocksRequest request = UncacheStaleBlocksRequest.newBuilder().build(); + UncacheStaleBlocksResponse response = null; + try { + response = admin.uncacheStaleBlocks(controller, request); + } catch (ServiceException se) { + throw getRemoteException(se); + } + return response.getUncachedFilesMap(); + } + /** * Get the list of region info from a GetOnlineRegionResponse * @param proto the GetOnlineRegionResponse diff --git a/hbase-protocol-shaded/src/main/protobuf/server/region/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/server/region/Admin.proto index 308b1a8b6d62..1b55577ac3ae 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/region/Admin.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/region/Admin.proto @@ -336,6 +336,14 @@ message ClearSlowLogResponses { required bool is_cleaned = 1; } +message UncacheStaleBlocksRequest { +} + +message UncacheStaleBlocksResponse { + map uncached_files = 1; +} + + service AdminService { rpc GetRegionInfo(GetRegionInfoRequest) returns(GetRegionInfoResponse); @@ -415,4 +423,7 @@ service AdminService { rpc GetCachedFilesList(GetCachedFilesListRequest) returns(GetCachedFilesListResponse); + rpc UncacheStaleBlocks(UncacheStaleBlocksRequest) + returns(UncacheStaleBlocksResponse); + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 57f71b31894e..ba737ecfc08b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -2146,6 +2146,5 @@ public Optional getBlockSize(BlockCacheKey key) { } else { return Optional.of(entry.getOnDiskSizeWithHeader()); } - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 1da8e03d179e..4d9e12eb31d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -184,6 +184,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest; @@ -3402,6 +3404,12 @@ public HBaseProtos.LogEntry getLogEntries(RpcController controller, throw new ServiceException("Invalid request params"); } + @Override + public UncacheStaleBlocksResponse uncacheStaleBlocks(RpcController controller, + UncacheStaleBlocksRequest request) throws ServiceException { + throw new ServiceException(new DoNotRetryIOException("Unsupported method on master")); + } + private MasterProtos.BalancerDecisionsResponse getBalancerDecisions(MasterProtos.BalancerDecisionsRequest request) { final NamedQueueRecorder namedQueueRecorder = this.server.getNamedQueueRecorder(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index dfb8e2a204fe..4653e5180ae4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -58,9 +58,12 @@ import java.util.TimerTask; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -1659,6 +1662,43 @@ public RegionLoad createRegionLoad(final String encodedRegionName) throws IOExce return r != null ? createRegionLoad(r, null, null) : null; } + public Map uncacheStaleBlocks() { + Map> fullyCachedFiles = + this.getBlockCache().flatMap(BlockCache::getFullyCachedFiles).orElse(Collections.emptyMap()); + Map evictedFilesWithStaleBlocks = new ConcurrentHashMap<>(); + + ExecutorService executor = Executors.newFixedThreadPool(6); + + List> tasks = new ArrayList<>(); + + fullyCachedFiles.forEach((fileName, value) -> { + Callable task = () -> { + HRegion regionOnServer = getRegion(value.getFirst()); + int blocksEvicted = (regionOnServer == null || !regionOnServer.isAvailable()) + ? this.getBlockCache().get().evictBlocksByHfileName(fileName) + : 0; + evictedFilesWithStaleBlocks.put(fileName, blocksEvicted); + LOG.info( + "Uncached {} blocks belonging to the file {} as the region {} " + + "is not served by the region server {} anymore.", + blocksEvicted, fileName, value.getFirst(), this.getServerName()); + return null; + }; + tasks.add(task); + }); + + try { + executor.invokeAll(tasks); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Thread interrupted while processing tasks for uncaching stale blocks: {}", + e.getMessage()); + } finally { + executor.shutdown(); + } + return evictedFilesWithStaleBlocks; + } + /** * Inner class that runs on a long period checking if regions need compaction. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 05d7c2e56055..ba9b50676337 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -192,6 +192,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry; @@ -3964,4 +3966,13 @@ public GetCachedFilesListResponse getCachedFilesList(RpcController controller, }); return responseBuilder.addAllCachedFiles(fullyCachedFiles).build(); } + + @Override + public UncacheStaleBlocksResponse uncacheStaleBlocks(RpcController controller, + UncacheStaleBlocksRequest request) throws ServiceException { + UncacheStaleBlocksResponse.Builder responseBuilder = UncacheStaleBlocksResponse.newBuilder(); + Map evictedFilesWithStaleBlocks = new HashMap<>(server.uncacheStaleBlocks()); + responseBuilder.putAllUncachedFiles(evictedFilesWithStaleBlocks); + return responseBuilder.build(); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCleanBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCleanBucketCache.java new file mode 100644 index 000000000000..fee10161c904 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCleanBucketCache.java @@ -0,0 +1,170 @@ +/* + * 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.io.hfile; + +import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.StartTestingClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ IOTests.class, MediumTests.class }) +public class TestCleanBucketCache { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCleanBucketCache.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + Path testDir; + MiniZooKeeperCluster zkCluster; + SingleProcessHBaseCluster cluster; + StartTestingClusterOption option = + StartTestingClusterOption.builder().numRegionServers(2).build(); + + @Before + public void setup() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + testDir = TEST_UTIL.getDataTestDir(); + TEST_UTIL.getTestFileSystem().mkdirs(testDir); + + conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true); + conf.set(BUCKET_CACHE_IOENGINE_KEY, "file:" + testDir + "/bucket.cache"); + conf.setInt("hbase.bucketcache.size", 400); + conf.set("hbase.bucketcache.persistent.path", testDir + "/bucket.persistence"); + conf.setLong(CacheConfig.BUCKETCACHE_PERSIST_INTERVAL_KEY, 100); + conf.setBoolean(CacheConfig.EVICT_BLOCKS_ON_CLOSE_KEY, true); + zkCluster = TEST_UTIL.startMiniZKCluster(); + cluster = TEST_UTIL.startMiniHBaseCluster(option); + cluster.setConf(conf); + } + + @Test + public void testCleanBucketCacheRegionClose() throws Exception { + // Write to table and flush + TableName tableRegionClose = writeDataToTable(); + + HRegionServer regionServingRS = + cluster.getRegionServer(1).getRegions(tableRegionClose).size() == 1 + ? cluster.getRegionServer(1) + : cluster.getRegionServer(0); + + assertTrue(regionServingRS.getBlockCache().isPresent()); + long oldUsedCacheSize = + regionServingRS.getBlockCache().get().getBlockCaches()[1].getCurrentSize(); + long blocksCount = regionServingRS.getBlockCache().get().getBlockCount(); + assertNotEquals(0, blocksCount); + + HRegion r = regionServingRS.getRegions(tableRegionClose).get(0); + r.close(false, false, true); + long newUsedCacheSize = + regionServingRS.getBlockCache().get().getBlockCaches()[1].getCurrentSize(); + assertEquals(oldUsedCacheSize, newUsedCacheSize); + assertNotEquals(0, regionServingRS.getBlockCache().get().getBlockCaches()[1].getBlockCount()); + assertTrue(r.isClosed()); + + Admin admin = TEST_UTIL.getAdmin(); + Map response = admin.uncacheStaleBlocks(regionServingRS.getServerName()); + + long newCacheSize = regionServingRS.getBlockCache().get().getBlockCaches()[1].getCurrentSize(); + assertTrue(newCacheSize < newUsedCacheSize); + response.values().forEach(value -> assertEquals(blocksCount, value.intValue())); + } + + @Test + public void testCleanBucketCacheRegionAvailable() throws Exception { + // Write to table and flush + TableName tableRegionClose = writeDataToTable(); + + HRegionServer regionServingRS = + cluster.getRegionServer(1).getRegions(tableRegionClose).size() == 1 + ? cluster.getRegionServer(1) + : cluster.getRegionServer(0); + + assertTrue(regionServingRS.getBlockCache().isPresent()); + long oldUsedCacheSize = + regionServingRS.getBlockCache().get().getBlockCaches()[1].getCurrentSize(); + long blocksCount = regionServingRS.getBlockCache().get().getBlockCount(); + assertNotEquals(0, blocksCount); + + Admin admin = TEST_UTIL.getAdmin(); + Map response = admin.uncacheStaleBlocks(regionServingRS.getServerName()); + assertEquals(regionServingRS.getBlockCache().get().getBlockCaches()[1].getCurrentSize(), + oldUsedCacheSize); + response.values().forEach(value -> assertEquals(0, value.intValue())); + } + + public TableName writeDataToTable() throws IOException, InterruptedException { + TableName tableName = TableName.valueOf("table1"); + byte[] row0 = Bytes.toBytes("row1"); + byte[] family = Bytes.toBytes("family"); + byte[] qf1 = Bytes.toBytes("qf1"); + byte[] value1 = Bytes.toBytes("value1"); + + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build(); + Table table = TEST_UTIL.createTable(td, null); + try { + // put data + Put put0 = new Put(row0); + put0.addColumn(family, qf1, 1, value1); + table.put(put0); + TEST_UTIL.flush(tableName); + } finally { + Thread.sleep(1500); + } + assertEquals(1, cluster.getRegions(tableName).size()); + return tableName; + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + TEST_UTIL.cleanupDataTestDirOnTestFS(String.valueOf(testDir)); + if (zkCluster != null) { + zkCluster.shutdown(); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index a7164a6fab64..5263ccab7b80 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -117,6 +117,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UncacheStaleBlocksResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; @@ -685,6 +687,12 @@ public HBaseProtos.LogEntry getLogEntries(RpcController controller, return null; } + @Override + public UncacheStaleBlocksResponse uncacheStaleBlocks(RpcController controller, + UncacheStaleBlocksRequest request) throws ServiceException { + return null; + } + @Override public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(RpcController controller, GetSpaceQuotaSnapshotsRequest request) throws ServiceException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java index 35c868413e19..af7ead196699 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java @@ -974,6 +974,11 @@ public void flushMasterStore() throws IOException { admin.flushMasterStore(); } + @Override + public Map uncacheStaleBlocks(ServerName serverName) throws IOException { + return admin.uncacheStaleBlocks(serverName); + } + @Override public List getCachedFilesList(ServerName serverName) throws IOException { return admin.getCachedFilesList(serverName); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java index 0eff84bba7c8..bd6cf2627766 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java @@ -1355,6 +1355,11 @@ public List getCachedFilesList(ServerName serverName) throws IOException throw new NotImplementedException("getCachedFilesList not supported in ThriftAdmin"); } + @Override + public Map uncacheStaleBlocks(ServerName serverName) throws IOException { + throw new NotImplementedException("uncacheStaleBlocks not supported in ThriftAdmin"); + } + @Override public boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures) throws IOException {