From 4e0c6c7adfd673a62cd7557e226504487976fd46 Mon Sep 17 00:00:00 2001 From: wenwj0 <36262341+wenwj0@users.noreply.github.com> Date: Tue, 14 Jun 2022 14:23:27 +0800 Subject: [PATCH] Revert "HBASE-26342 Support custom paths of independent configuration and pool for hfile cleaner (#4403)" This reverts commit 87c32f23859e97613ca6b125073d5f12b9c87d49. --- .../apache/hadoop/hbase/master/HMaster.java | 99 +++------------ .../hbase/master/MasterRpcServices.java | 5 +- .../hbase/master/cleaner/CleanerChore.java | 53 +------- .../hbase/master/cleaner/DirScanPool.java | 11 +- .../hbase/master/cleaner/HFileCleaner.java | 23 +--- .../hbase/master/cleaner/LogCleaner.java | 2 +- .../master/TestMasterChoreScheduled.java | 7 +- .../cleaner/TestCleanerClearHFiles.java | 120 ------------------ 8 files changed, 33 insertions(+), 287 deletions(-) delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerClearHFiles.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index e4f2391771ae..0fb182c4a90d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -20,7 +20,6 @@ import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK; import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS; import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK; -import static org.apache.hadoop.hbase.master.cleaner.HFileCleaner.CUSTOM_POOL_SIZE; import static org.apache.hadoop.hbase.util.DNS.MASTER_HOSTNAME_KEY; import com.google.errorprone.annotations.RestrictedApi; @@ -38,7 +37,6 @@ import java.util.Comparator; import java.util.EnumSet; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -80,7 +78,6 @@ import org.apache.hadoop.hbase.PleaseRestartMasterException; import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.ReplicationPeerNotFoundException; -import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.ServerMetrics; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerTask; @@ -381,18 +378,12 @@ public class HMaster extends HBaseServerBase implements Maste private HbckChore hbckChore; CatalogJanitor catalogJanitorChore; + // Threadpool for scanning the archive directory, used by the HFileCleaner + private DirScanPool hfileCleanerPool; // Threadpool for scanning the Old logs directory, used by the LogCleaner private DirScanPool logCleanerPool; private LogCleaner logCleaner; - // HFile cleaners for the custom hfile archive paths and the default archive path - // The archive path cleaner is the first element - private List hfileCleaners = new ArrayList<>(); - // The hfile cleaner paths, including custom paths and the default archive path - private List hfileCleanerPaths = new ArrayList<>(); - // The shared hfile cleaner pool for the custom archive paths - private DirScanPool sharedHFileCleanerPool; - // The exclusive hfile cleaner pool for scanning the archive directory - private DirScanPool exclusiveHFileCleanerPool; + private HFileCleaner hfileCleaner; private ReplicationBarrierCleaner replicationBarrierCleaner; private MobFileCleanerChore mobFileCleanerChore; private MobFileCompactionChore mobFileCompactionChore; @@ -1167,18 +1158,11 @@ private void finishActiveMasterInitialization(MonitoredTask status) (EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f)); this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime(); configurationManager.registerObserver(this.balancer); + configurationManager.registerObserver(this.hfileCleanerPool); configurationManager.registerObserver(this.logCleanerPool); + configurationManager.registerObserver(this.hfileCleaner); configurationManager.registerObserver(this.logCleaner); configurationManager.registerObserver(this.regionsRecoveryConfigManager); - configurationManager.registerObserver(this.exclusiveHFileCleanerPool); - if (this.sharedHFileCleanerPool != null) { - configurationManager.registerObserver(this.sharedHFileCleanerPool); - } - if (this.hfileCleaners != null) { - for (HFileCleaner cleaner : hfileCleaners) { - configurationManager.registerObserver(cleaner); - } - } // Set master as 'initialized'. setInitialized(true); @@ -1455,8 +1439,8 @@ public boolean isCatalogJanitorEnabled() { boolean isCleanerChoreEnabled() { boolean hfileCleanerFlag = true, logCleanerFlag = true; - if (getHFileCleaner() != null) { - hfileCleanerFlag = getHFileCleaner().getEnabled(); + if (hfileCleaner != null) { + hfileCleanerFlag = hfileCleaner.getEnabled(); } if (logCleaner != null) { @@ -1555,47 +1539,13 @@ executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_MERGE_O getMasterWalManager().getOldLogDir(), logCleanerPool, params); getChoreService().scheduleChore(logCleaner); + // start the hfile archive cleaner thread Path archiveDir = HFileArchiveUtil.getArchivePath(conf); - - // Create custom archive hfile cleaners - String[] paths = conf.getStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS); - // todo: handle the overlap issues for the custom paths - - if (paths != null && paths.length > 0) { - if (conf.getStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS) == null) { - Set cleanerClasses = new HashSet<>(); - String[] cleaners = conf.getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS); - if (cleaners != null) { - Collections.addAll(cleanerClasses, cleaners); - } - conf.setStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS, - cleanerClasses.toArray(new String[cleanerClasses.size()])); - LOG.info("Archive custom cleaner paths: {}, plugins: {}", Arrays.asList(paths), - cleanerClasses); - } - // share the hfile cleaner pool in custom paths - sharedHFileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf.get(CUSTOM_POOL_SIZE, "6")); - for (int i = 0; i < paths.length; i++) { - Path path = new Path(paths[i].trim()); - HFileCleaner cleaner = - new HFileCleaner("ArchiveCustomHFileCleaner-" + path.getName(), cleanerInterval, this, - conf, getMasterFileSystem().getFileSystem(), new Path(archiveDir, path), - HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS, sharedHFileCleanerPool, params, null); - hfileCleaners.add(cleaner); - hfileCleanerPaths.add(path); - } - } - - // Create the whole archive dir cleaner thread pool - exclusiveHFileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf); - hfileCleaners.add(0, - new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem().getFileSystem(), - archiveDir, exclusiveHFileCleanerPool, params, hfileCleanerPaths)); - hfileCleanerPaths.add(0, archiveDir); - // Schedule all the hfile cleaners - for (HFileCleaner hFileCleaner : hfileCleaners) { - getChoreService().scheduleChore(hFileCleaner); - } + // Create archive cleaner thread pool + hfileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf); + this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, + getMasterFileSystem().getFileSystem(), archiveDir, hfileCleanerPool, params); + getChoreService().scheduleChore(hfileCleaner); // Regions Reopen based on very high storeFileRefCount is considered enabled // only if hbase.regions.recovery.store.file.ref.count has value > 0 @@ -1643,18 +1593,14 @@ protected void stopServiceThreads() { } stopChoreService(); stopExecutorService(); - if (exclusiveHFileCleanerPool != null) { - exclusiveHFileCleanerPool.shutdownNow(); - exclusiveHFileCleanerPool = null; + if (hfileCleanerPool != null) { + hfileCleanerPool.shutdownNow(); + hfileCleanerPool = null; } if (logCleanerPool != null) { logCleanerPool.shutdownNow(); logCleanerPool = null; } - if (sharedHFileCleanerPool != null) { - sharedHFileCleanerPool.shutdownNow(); - sharedHFileCleanerPool = null; - } if (maintenanceRegionServer != null) { maintenanceRegionServer.getRegionServer().stop(HBASE_MASTER_CLEANER_INTERVAL); } @@ -1789,12 +1735,7 @@ protected void stopChores() { shutdownChore(clusterStatusPublisherChore); shutdownChore(snapshotQuotaChore); shutdownChore(logCleaner); - if (hfileCleaners != null) { - for (ScheduledChore chore : hfileCleaners) { - chore.shutdown(); - } - hfileCleaners = null; - } + shutdownChore(hfileCleaner); shutdownChore(replicationBarrierCleaner); shutdownChore(snapshotCleanerChore); shutdownChore(hbckChore); @@ -3269,11 +3210,7 @@ public static void main(String[] args) { } public HFileCleaner getHFileCleaner() { - return this.hfileCleaners.get(0); - } - - public List getHFileCleaners() { - return this.hfileCleaners; + return this.hfileCleaner; } public LogCleaner getLogCleaner() { 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 fd1826fd9cad..dd2a18f5ae3a 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 @@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.master.assignment.RegionStateNode; import org.apache.hadoop.hbase.master.assignment.RegionStates; -import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.janitor.MetaFixer; import org.apache.hadoop.hbase.master.locking.LockProcedure; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; @@ -875,9 +874,7 @@ public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController c, boolean prevValue = server.getLogCleaner().getEnabled() && server.getHFileCleaner().getEnabled(); server.getLogCleaner().setEnabled(req.getOn()); - for (HFileCleaner hFileCleaner : server.getHFileCleaners()) { - hFileCleaner.setEnabled(req.getOn()); - } + server.getHFileCleaner().setEnabled(req.getOn()); return SetCleanerChoreRunningResponse.newBuilder().setPrevValue(prevValue).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java index 801d7939a789..80908e1e0509 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.master.cleaner; -import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -80,11 +79,10 @@ public abstract class CleanerChore extends Schedu protected final Map params; private final AtomicBoolean enabled = new AtomicBoolean(true); protected List cleanersChain; - protected List excludeDirs; public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf, FileSystem fs, Path oldFileDir, String confKey, DirScanPool pool) { - this(name, sleepPeriod, s, conf, fs, oldFileDir, confKey, pool, null, null); + this(name, sleepPeriod, s, conf, fs, oldFileDir, confKey, pool, null); } /** @@ -99,8 +97,7 @@ public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Confi * @param params members could be used in cleaner */ public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf, - FileSystem fs, Path oldFileDir, String confKey, DirScanPool pool, Map params, - List excludePaths) { + FileSystem fs, Path oldFileDir, String confKey, DirScanPool pool, Map params) { super(name, s, sleepPeriod); Preconditions.checkNotNull(pool, "Chore's pool can not be null"); @@ -109,19 +106,6 @@ public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Confi this.oldFileDir = oldFileDir; this.conf = conf; this.params = params; - if (excludePaths != null && !excludePaths.isEmpty()) { - excludeDirs = new ArrayList<>(excludePaths.size()); - for (Path path : excludePaths) { - StringBuilder dirPart = new StringBuilder(path.toString()); - if (!path.toString().endsWith("/")) { - dirPart.append("/"); - } - excludeDirs.add(dirPart.toString()); - } - } - if (excludeDirs != null) { - LOG.info("Cleaner {} excludes sub dirs: {}", name, excludeDirs); - } initCleanerChain(confKey); } @@ -435,11 +419,9 @@ private void traverseAndDelete(Path dir, boolean root, CompletableFuture { - if (!shouldExclude(subDir)) { - CompletableFuture subFuture = new CompletableFuture<>(); - pool.execute(() -> traverseAndDelete(subDir.getPath(), false, subFuture)); - futures.add(subFuture); - } + CompletableFuture subFuture = new CompletableFuture<>(); + pool.execute(() -> traverseAndDelete(subDir.getPath(), false, subFuture)); + futures.add(subFuture); }); } @@ -469,34 +451,11 @@ private void traverseAndDelete(Path dir, boolean root, CompletableFuture largeFileQueue; @@ -128,13 +117,8 @@ public HFileCleaner(final int period, final Stoppable stopper, Configuration con public HFileCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs, Path directory, DirScanPool pool, Map params) { this("HFileCleaner", period, stopper, conf, fs, directory, MASTER_HFILE_CLEANER_PLUGINS, pool, - params, null); - } + params); - public HFileCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs, - Path directory, DirScanPool pool, Map params, List excludePaths) { - this("HFileCleaner", period, stopper, conf, fs, directory, MASTER_HFILE_CLEANER_PLUGINS, pool, - params, excludePaths); } /** @@ -150,9 +134,8 @@ public HFileCleaner(final int period, final Stoppable stopper, Configuration con * @param params params could be used in subclass of BaseHFileCleanerDelegate */ public HFileCleaner(String name, int period, Stoppable stopper, Configuration conf, FileSystem fs, - Path directory, String confKey, DirScanPool pool, Map params, - List excludePaths) { - super(name, period, stopper, conf, fs, directory, confKey, pool, params, excludePaths); + Path directory, String confKey, DirScanPool pool, Map params) { + super(name, period, stopper, conf, fs, directory, confKey, pool, params); throttlePoint = conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD); largeQueueInitSize = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java index ac0a98801c15..90e498583c0c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java @@ -75,7 +75,7 @@ public class LogCleaner extends CleanerChore public LogCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs, Path oldLogDir, DirScanPool pool, Map params) { super("LogsCleaner", period, stopper, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS, - pool, params, null); + pool, params); this.pendingDelete = new LinkedBlockingQueue<>(); int size = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE); this.oldWALsCleaner = createOldWalsCleaner(size); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java index 68ce244d1ccc..c7b26fe01bc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.master; import java.lang.reflect.Field; -import java.util.ArrayList; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.ScheduledChore; @@ -64,7 +63,7 @@ public static void tearDown() throws Exception { } @Test - public void testDefaultScheduledChores() throws Exception { + public void testDefaultScheduledChores() { // test if logCleaner chore is scheduled by default in HMaster init TestChoreField logCleanerTestChoreField = new TestChoreField<>(); LogCleaner logCleaner = logCleanerTestChoreField.getChoreObj("logCleaner"); @@ -72,9 +71,7 @@ public void testDefaultScheduledChores() throws Exception { // test if hfileCleaner chore is scheduled by default in HMaster init TestChoreField hFileCleanerTestChoreField = new TestChoreField<>(); - Field masterField = HMaster.class.getDeclaredField("hfileCleaners"); - masterField.setAccessible(true); - HFileCleaner hFileCleaner = ((ArrayList) masterField.get(hMaster)).get(0); + HFileCleaner hFileCleaner = hFileCleanerTestChoreField.getChoreObj("hfileCleaner"); hFileCleanerTestChoreField.testIfChoreScheduled(hFileCleaner); // test if replicationBarrierCleaner chore is scheduled by default in HMaster init diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerClearHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerClearHFiles.java deleted file mode 100644 index c747a14be1ae..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerClearHFiles.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.master.cleaner; - -import static org.apache.hadoop.hbase.master.HMaster.HBASE_MASTER_CLEANER_INTERVAL; -import static org.apache.hadoop.hbase.master.cleaner.HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS; - -import java.io.FileNotFoundException; -import java.io.IOException; -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.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -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.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.HFileArchiveUtil; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; - -@Category(LargeTests.class) -public class TestCleanerClearHFiles { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestCleanerClearHFiles.class); - - @Rule - public TestName name = new TestName(); - - private static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); - private static Configuration conf = TEST_UTIL.getConfiguration(); - private static Admin admin = null; - - private static final byte[] COLUMN_FAMILY = Bytes.toBytes("CF"); - - private static final String TABLE1 = "table1"; - private static final String TABLE2 = "table2"; - private static final String DEFAULT_ARCHIVE_SUBDIRS_PREFIX = "data/default/"; - - @BeforeClass - public static void setupBeforeClass() throws Exception { - conf.setStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS, - DEFAULT_ARCHIVE_SUBDIRS_PREFIX + TABLE1); - conf.setStrings(HFILE_CLEANER_CUSTOM_PATHS_PLUGINS, HFileLinkCleaner.class.getName()); - - conf.setInt(TimeToLiveHFileCleaner.TTL_CONF_KEY, 10); - conf.setInt(HBASE_MASTER_CLEANER_INTERVAL, 20000); - - TEST_UTIL.startMiniCluster(); - admin = TEST_UTIL.getAdmin(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testClearArchive() throws Exception { - DistributedFileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem(); - Table table1 = createTable(TEST_UTIL, TableName.valueOf(TABLE1)); - Table table2 = createTable(TEST_UTIL, TableName.valueOf(TABLE2)); - - admin.disableTable(table1.getName()); - admin.deleteTable(table1.getName()); - admin.disableTable(table2.getName()); - admin.deleteTable(table2.getName()); - - Path archiveDir = HFileArchiveUtil.getArchivePath(conf); - Path archiveTable1Path = new Path(archiveDir, DEFAULT_ARCHIVE_SUBDIRS_PREFIX + TABLE1); - Path archiveTable2Path = new Path(archiveDir, DEFAULT_ARCHIVE_SUBDIRS_PREFIX + TABLE2); - - TEST_UTIL.waitFor(10000, () -> !notExistOrEmptyDir(archiveTable1Path, fs) - && !notExistOrEmptyDir(archiveTable2Path, fs)); - - TEST_UTIL.waitFor(30000, - () -> notExistOrEmptyDir(archiveTable1Path, fs) && notExistOrEmptyDir(archiveTable2Path, fs)); - } - - private boolean notExistOrEmptyDir(Path dir, DistributedFileSystem fs) { - try { - return fs.listStatus(dir).length == 0; - } catch (Exception e) { - return e instanceof FileNotFoundException; - } - } - - private Table createTable(HBaseTestingUtil util, TableName tableName) throws IOException { - TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName) - .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_FAMILY).build()).build(); - return util.createTable(td, null); - } -}