View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.master;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.util.ArrayList;
24  import java.util.HashSet;
25  import java.util.List;
26  import java.util.Set;
27  import java.util.concurrent.locks.Lock;
28  import java.util.concurrent.locks.ReentrantLock;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FileStatus;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.fs.PathFilter;
38  import org.apache.hadoop.fs.permission.FsPermission;
39  import org.apache.hadoop.hbase.ClusterId;
40  import org.apache.hadoop.hbase.TableName;
41  import org.apache.hadoop.hbase.HColumnDescriptor;
42  import org.apache.hadoop.hbase.HConstants;
43  import org.apache.hadoop.hbase.HRegionInfo;
44  import org.apache.hadoop.hbase.HTableDescriptor;
45  import org.apache.hadoop.hbase.InvalidFamilyOperationException;
46  import org.apache.hadoop.hbase.RemoteExceptionHandler;
47  import org.apache.hadoop.hbase.Server;
48  import org.apache.hadoop.hbase.ServerName;
49  import org.apache.hadoop.hbase.backup.HFileArchiver;
50  import org.apache.hadoop.hbase.exceptions.DeserializationException;
51  import org.apache.hadoop.hbase.fs.HFileSystem;
52  import org.apache.hadoop.hbase.mob.MobConstants;
53  import org.apache.hadoop.hbase.mob.MobUtils;
54  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
55  import org.apache.hadoop.hbase.regionserver.HRegion;
56  import org.apache.hadoop.hbase.wal.DefaultWALProvider;
57  import org.apache.hadoop.hbase.wal.WALSplitter;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
60  import org.apache.hadoop.hbase.util.FSTableDescriptors;
61  import org.apache.hadoop.hbase.util.FSUtils;
62  
63  /**
64   * This class abstracts a bunch of operations the HMaster needs to interact with
65   * the underlying file system, including splitting log files, checking file
66   * system status, etc.
67   */
68  @InterfaceAudience.Private
69  public class MasterFileSystem {
70    private static final Log LOG = LogFactory.getLog(MasterFileSystem.class.getName());
71    // HBase configuration
72    Configuration conf;
73    // master status
74    Server master;
75    // metrics for master
76    private final MetricsMasterFileSystem metricsMasterFilesystem = new MetricsMasterFileSystem();
77    // Persisted unique cluster ID
78    private ClusterId clusterId;
79    // Keep around for convenience.
80    private final FileSystem fs;
81    // Is the fileystem ok?
82    private volatile boolean fsOk = true;
83    // The Path to the old logs dir
84    private final Path oldLogDir;
85    // root hbase directory on the FS
86    private final Path rootdir;
87    // hbase temp directory used for table construction and deletion
88    private final Path tempdir;
89    // create the split log lock
90    final Lock splitLogLock = new ReentrantLock();
91    final boolean distributedLogReplay;
92    final SplitLogManager splitLogManager;
93    private final MasterServices services;
94  
95    final static PathFilter META_FILTER = new PathFilter() {
96      @Override
97      public boolean accept(Path p) {
98        return DefaultWALProvider.isMetaFile(p);
99      }
100   };
101 
102   final static PathFilter NON_META_FILTER = new PathFilter() {
103     @Override
104     public boolean accept(Path p) {
105       return !DefaultWALProvider.isMetaFile(p);
106     }
107   };
108 
109   public MasterFileSystem(Server master, MasterServices services)
110   throws IOException {
111     this.conf = master.getConfiguration();
112     this.master = master;
113     this.services = services;
114     // Set filesystem to be that of this.rootdir else we get complaints about
115     // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
116     // default localfs.  Presumption is that rootdir is fully-qualified before
117     // we get to here with appropriate fs scheme.
118     this.rootdir = FSUtils.getRootDir(conf);
119     this.tempdir = new Path(this.rootdir, HConstants.HBASE_TEMP_DIRECTORY);
120     // Cover both bases, the old way of setting default fs and the new.
121     // We're supposed to run on 0.20 and 0.21 anyways.
122     this.fs = this.rootdir.getFileSystem(conf);
123     FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
124     // make sure the fs has the same conf
125     fs.setConf(conf);
126     // setup the filesystem variable
127     // set up the archived logs path
128     this.oldLogDir = createInitialFileSystemLayout();
129     HFileSystem.addLocationsOrderInterceptor(conf);
130     this.splitLogManager =
131         new SplitLogManager(master, master.getConfiguration(), master, services,
132             master.getServerName());
133     this.distributedLogReplay = this.splitLogManager.isLogReplaying();
134   }
135 
136   /**
137    * Create initial layout in filesystem.
138    * <ol>
139    * <li>Check if the meta region exists and is readable, if not create it.
140    * Create hbase.version and the hbase:meta directory if not one.
141    * </li>
142    * <li>Create a log archive directory for RS to put archived logs</li>
143    * </ol>
144    * Idempotent.
145    */
146   private Path createInitialFileSystemLayout() throws IOException {
147     // check if the root directory exists
148     checkRootDir(this.rootdir, conf, this.fs);
149 
150     // check if temp directory exists and clean it
151     checkTempDir(this.tempdir, conf, this.fs);
152 
153     Path oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
154 
155     // Make sure the region servers can archive their old logs
156     if(!this.fs.exists(oldLogDir)) {
157       this.fs.mkdirs(oldLogDir);
158     }
159 
160     return oldLogDir;
161   }
162 
163   public FileSystem getFileSystem() {
164     return this.fs;
165   }
166 
167   /**
168    * Get the directory where old logs go
169    * @return the dir
170    */
171   public Path getOldLogDir() {
172     return this.oldLogDir;
173   }
174 
175   /**
176    * Checks to see if the file system is still accessible.
177    * If not, sets closed
178    * @return false if file system is not available
179    */
180   public boolean checkFileSystem() {
181     if (this.fsOk) {
182       try {
183         FSUtils.checkFileSystemAvailable(this.fs);
184         FSUtils.checkDfsSafeMode(this.conf);
185       } catch (IOException e) {
186         master.abort("Shutting down HBase cluster: file system not available", e);
187         this.fsOk = false;
188       }
189     }
190     return this.fsOk;
191   }
192 
193   /**
194    * @return HBase root dir.
195    */
196   public Path getRootDir() {
197     return this.rootdir;
198   }
199 
200   /**
201    * @return HBase temp dir.
202    */
203   public Path getTempDir() {
204     return this.tempdir;
205   }
206 
207   /**
208    * @return The unique identifier generated for this cluster
209    */
210   public ClusterId getClusterId() {
211     return clusterId;
212   }
213 
214   /**
215    * Inspect the log directory to find dead servers which need recovery work
216    * @return A set of ServerNames which aren't running but still have WAL files left in file system
217    */
218   Set<ServerName> getFailedServersFromLogFolders() {
219     boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors",
220         WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
221 
222     Set<ServerName> serverNames = new HashSet<ServerName>();
223     Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
224 
225     do {
226       if (master.isStopped()) {
227         LOG.warn("Master stopped while trying to get failed servers.");
228         break;
229       }
230       try {
231         if (!this.fs.exists(logsDirPath)) return serverNames;
232         FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
233         // Get online servers after getting log folders to avoid log folder deletion of newly
234         // checked in region servers . see HBASE-5916
235         Set<ServerName> onlineServers = ((HMaster) master).getServerManager().getOnlineServers()
236             .keySet();
237 
238         if (logFolders == null || logFolders.length == 0) {
239           LOG.debug("No log files to split, proceeding...");
240           return serverNames;
241         }
242         for (FileStatus status : logFolders) {
243           FileStatus[] curLogFiles = FSUtils.listStatus(this.fs, status.getPath(), null);
244           if (curLogFiles == null || curLogFiles.length == 0) {
245             // Empty log folder. No recovery needed
246             continue;
247           }
248           final ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(
249               status.getPath());
250           if (null == serverName) {
251             LOG.warn("Log folder " + status.getPath() + " doesn't look like its name includes a " +
252                 "region server name; leaving in place. If you see later errors about missing " +
253                 "write ahead logs they may be saved in this location.");
254           } else if (!onlineServers.contains(serverName)) {
255             LOG.info("Log folder " + status.getPath() + " doesn't belong "
256                 + "to a known region server, splitting");
257             serverNames.add(serverName);
258           } else {
259             LOG.info("Log folder " + status.getPath() + " belongs to an existing region server");
260           }
261         }
262         retrySplitting = false;
263       } catch (IOException ioe) {
264         LOG.warn("Failed getting failed servers to be recovered.", ioe);
265         if (!checkFileSystem()) {
266           LOG.warn("Bad Filesystem, exiting");
267           Runtime.getRuntime().halt(1);
268         }
269         try {
270           if (retrySplitting) {
271             Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000));
272           }
273         } catch (InterruptedException e) {
274           LOG.warn("Interrupted, aborting since cannot return w/o splitting");
275           Thread.currentThread().interrupt();
276           retrySplitting = false;
277           Runtime.getRuntime().halt(1);
278         }
279       }
280     } while (retrySplitting);
281 
282     return serverNames;
283   }
284 
285   public void splitLog(final ServerName serverName) throws IOException {
286     Set<ServerName> serverNames = new HashSet<ServerName>();
287     serverNames.add(serverName);
288     splitLog(serverNames);
289   }
290 
291   /**
292    * Specialized method to handle the splitting for meta WAL
293    * @param serverName
294    * @throws IOException
295    */
296   public void splitMetaLog(final ServerName serverName) throws IOException {
297     Set<ServerName> serverNames = new HashSet<ServerName>();
298     serverNames.add(serverName);
299     splitMetaLog(serverNames);
300   }
301 
302   /**
303    * Specialized method to handle the splitting for meta WAL
304    * @param serverNames
305    * @throws IOException
306    */
307   public void splitMetaLog(final Set<ServerName> serverNames) throws IOException {
308     splitLog(serverNames, META_FILTER);
309   }
310 
311   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification=
312       "We only release this lock when we set it. Updates to code that uses it should verify use " +
313       "of the guard boolean.")
314   private List<Path> getLogDirs(final Set<ServerName> serverNames) throws IOException {
315     List<Path> logDirs = new ArrayList<Path>();
316     boolean needReleaseLock = false;
317     if (!this.services.isInitialized()) {
318       // during master initialization, we could have multiple places splitting a same wal
319       this.splitLogLock.lock();
320       needReleaseLock = true;
321     }
322     try {
323       for (ServerName serverName : serverNames) {
324         Path logDir = new Path(this.rootdir,
325             DefaultWALProvider.getWALDirectoryName(serverName.toString()));
326         Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
327         // Rename the directory so a rogue RS doesn't create more WALs
328         if (fs.exists(logDir)) {
329           if (!this.fs.rename(logDir, splitDir)) {
330             throw new IOException("Failed fs.rename for log split: " + logDir);
331           }
332           logDir = splitDir;
333           LOG.debug("Renamed region directory: " + splitDir);
334         } else if (!fs.exists(splitDir)) {
335           LOG.info("Log dir for server " + serverName + " does not exist");
336           continue;
337         }
338         logDirs.add(splitDir);
339       }
340     } finally {
341       if (needReleaseLock) {
342         this.splitLogLock.unlock();
343       }
344     }
345     return logDirs;
346   }
347 
348   /**
349    * Mark regions in recovering state when distributedLogReplay are set true
350    * @param serverName Failed region server whose wals to be replayed
351    * @param regions Set of regions to be recovered
352    * @throws IOException
353    */
354   public void prepareLogReplay(ServerName serverName, Set<HRegionInfo> regions) throws IOException {
355     if (!this.distributedLogReplay) {
356       return;
357     }
358     // mark regions in recovering state
359     if (regions == null || regions.isEmpty()) {
360       return;
361     }
362     this.splitLogManager.markRegionsRecovering(serverName, regions);
363   }
364 
365   public void splitLog(final Set<ServerName> serverNames) throws IOException {
366     splitLog(serverNames, NON_META_FILTER);
367   }
368 
369   /**
370    * Wrapper function on {@link SplitLogManager#removeStaleRecoveringRegions(Set)}
371    * @param failedServers
372    * @throws IOException
373    */
374   void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
375       throws IOException, InterruptedIOException {
376     this.splitLogManager.removeStaleRecoveringRegions(failedServers);
377   }
378 
379   /**
380    * This method is the base split method that splits WAL files matching a filter. Callers should
381    * pass the appropriate filter for meta and non-meta WALs.
382    * @param serverNames logs belonging to these servers will be split; this will rename the log
383    *                    directory out from under a soft-failed server
384    * @param filter
385    * @throws IOException
386    */
387   public void splitLog(final Set<ServerName> serverNames, PathFilter filter) throws IOException {
388     long splitTime = 0, splitLogSize = 0;
389     List<Path> logDirs = getLogDirs(serverNames);
390 
391     splitLogManager.handleDeadWorkers(serverNames);
392     splitTime = EnvironmentEdgeManager.currentTime();
393     splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter);
394     splitTime = EnvironmentEdgeManager.currentTime() - splitTime;
395 
396     if (this.metricsMasterFilesystem != null) {
397       if (filter == META_FILTER) {
398         this.metricsMasterFilesystem.addMetaWALSplit(splitTime, splitLogSize);
399       } else {
400         this.metricsMasterFilesystem.addSplit(splitTime, splitLogSize);
401       }
402     }
403   }
404 
405   /**
406    * Get the rootdir.  Make sure its wholesome and exists before returning.
407    * @param rd
408    * @param c
409    * @param fs
410    * @return hbase.rootdir (after checks for existence and bootstrapping if
411    * needed populating the directory with necessary bootup files).
412    * @throws IOException
413    */
414   @SuppressWarnings("deprecation")
415   private Path checkRootDir(final Path rd, final Configuration c,
416     final FileSystem fs)
417   throws IOException {
418     // If FS is in safe mode wait till out of it.
419     FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
420 
421     boolean isSecurityEnabled = "kerberos".equalsIgnoreCase(c.get("hbase.security.authentication"));
422     FsPermission rootDirPerms = new FsPermission(c.get("hbase.rootdir.perms", "700"));
423 
424     // Filesystem is good. Go ahead and check for hbase.rootdir.
425     try {
426       if (!fs.exists(rd)) {
427         if (isSecurityEnabled) {
428           fs.mkdirs(rd, rootDirPerms);
429         } else {
430           fs.mkdirs(rd);
431         }
432         // DFS leaves safe mode with 0 DNs when there are 0 blocks.
433         // We used to handle this by checking the current DN count and waiting until
434         // it is nonzero. With security, the check for datanode count doesn't work --
435         // it is a privileged op. So instead we adopt the strategy of the jobtracker
436         // and simply retry file creation during bootstrap indefinitely. As soon as
437         // there is one datanode it will succeed. Permission problems should have
438         // already been caught by mkdirs above.
439         FSUtils.setVersion(fs, rd, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
440           10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
441             HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
442       } else {
443         if (!fs.isDirectory(rd)) {
444           throw new IllegalArgumentException(rd.toString() + " is not a directory");
445         }
446         if (isSecurityEnabled && !rootDirPerms.equals(fs.getFileStatus(rd).getPermission())) {
447           // check whether the permission match
448           LOG.warn("Found rootdir permissions NOT matching expected \"hbase.rootdir.perms\" for "
449               + "rootdir=" + rd.toString() + " permissions=" + fs.getFileStatus(rd).getPermission()
450               + " and  \"hbase.rootdir.perms\" configured as "
451               + c.get("hbase.rootdir.perms", "700") + ". Automatically setting the permissions. You"
452               + " can change the permissions by setting \"hbase.rootdir.perms\" in hbase-site.xml "
453               + "and restarting the master");
454           fs.setPermission(rd, rootDirPerms);
455         }
456         // as above
457         FSUtils.checkVersion(fs, rd, true, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
458           10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
459             HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
460       }
461     } catch (DeserializationException de) {
462       LOG.fatal("Please fix invalid configuration for " + HConstants.HBASE_DIR, de);
463       IOException ioe = new IOException();
464       ioe.initCause(de);
465       throw ioe;
466     } catch (IllegalArgumentException iae) {
467       LOG.fatal("Please fix invalid configuration for "
468         + HConstants.HBASE_DIR + " " + rd.toString(), iae);
469       throw iae;
470     }
471     // Make sure cluster ID exists
472     if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
473         HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
474       FSUtils.setClusterId(fs, rd, new ClusterId(), c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
475     }
476     clusterId = FSUtils.getClusterId(fs, rd);
477 
478     // Make sure the meta region directory exists!
479     if (!FSUtils.metaRegionExists(fs, rd)) {
480       bootstrap(rd, c);
481     } else {
482       // Migrate table descriptor files if necessary
483       org.apache.hadoop.hbase.util.FSTableDescriptorMigrationToSubdir
484         .migrateFSTableDescriptorsIfNecessary(fs, rd);
485     }
486 
487     // Create tableinfo-s for hbase:meta if not already there.
488 
489     // meta table is a system table, so descriptors are predefined,
490     // we should get them from registry.
491     FSTableDescriptors fsd = new FSTableDescriptors(c, fs, rd);
492     fsd.createTableDescriptor(
493       new HTableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
494 
495     return rd;
496   }
497 
498   /**
499    * Make sure the hbase temp directory exists and is empty.
500    * NOTE that this method is only executed once just after the master becomes the active one.
501    */
502   private void checkTempDir(final Path tmpdir, final Configuration c, final FileSystem fs)
503       throws IOException {
504     // If the temp directory exists, clear the content (left over, from the previous run)
505     if (fs.exists(tmpdir)) {
506       // Archive table in temp, maybe left over from failed deletion,
507       // if not the cleaner will take care of them.
508       for (Path tabledir: FSUtils.getTableDirs(fs, tmpdir)) {
509         for (Path regiondir: FSUtils.getRegionDirs(fs, tabledir)) {
510           HFileArchiver.archiveRegion(fs, this.rootdir, tabledir, regiondir);
511         }
512       }
513       if (!fs.delete(tmpdir, true)) {
514         throw new IOException("Unable to clean the temp directory: " + tmpdir);
515       }
516     }
517 
518     // Create the temp directory
519     if (!fs.mkdirs(tmpdir)) {
520       throw new IOException("HBase temp directory '" + tmpdir + "' creation failure.");
521     }
522   }
523 
524   private static void bootstrap(final Path rd, final Configuration c)
525   throws IOException {
526     LOG.info("BOOTSTRAP: creating hbase:meta region");
527     try {
528       // Bootstrapping, make sure blockcache is off.  Else, one will be
529       // created here in bootstrap and it'll need to be cleaned up.  Better to
530       // not make it in first place.  Turn off block caching for bootstrap.
531       // Enable after.
532       HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
533       HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
534       setInfoFamilyCachingForMeta(metaDescriptor, false);
535       HRegion meta = HRegion.createHRegion(metaHRI, rd, c, metaDescriptor);
536       setInfoFamilyCachingForMeta(metaDescriptor, true);
537       HRegion.closeHRegion(meta);
538     } catch (IOException e) {
539       e = RemoteExceptionHandler.checkIOException(e);
540       LOG.error("bootstrap", e);
541       throw e;
542     }
543   }
544 
545   /**
546    * Enable in memory caching for hbase:meta
547    */
548   public static void setInfoFamilyCachingForMeta(final HTableDescriptor metaDescriptor,
549       final boolean b) {
550     for (HColumnDescriptor hcd: metaDescriptor.getColumnFamilies()) {
551       if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
552         hcd.setBlockCacheEnabled(b);
553         hcd.setInMemory(b);
554       }
555     }
556   }
557 
558   public void deleteRegion(HRegionInfo region) throws IOException {
559     HFileArchiver.archiveRegion(conf, fs, region);
560   }
561 
562   public void deleteTable(TableName tableName) throws IOException {
563     fs.delete(FSUtils.getTableDir(rootdir, tableName), true);
564   }
565 
566   /**
567    * Move the specified table to the hbase temp directory
568    * @param tableName Table name to move
569    * @return The temp location of the table moved
570    * @throws IOException in case of file-system failure
571    */
572   public Path moveTableToTemp(TableName tableName) throws IOException {
573     Path srcPath = FSUtils.getTableDir(rootdir, tableName);
574     Path tempPath = FSUtils.getTableDir(this.tempdir, tableName);
575 
576     // Ensure temp exists
577     if (!fs.exists(tempPath.getParent()) && !fs.mkdirs(tempPath.getParent())) {
578       throw new IOException("HBase temp directory '" + tempPath.getParent() + "' creation failure.");
579     }
580 
581     if (!fs.rename(srcPath, tempPath)) {
582       throw new IOException("Unable to move '" + srcPath + "' to temp '" + tempPath + "'");
583     }
584 
585     return tempPath;
586   }
587 
588   public void updateRegionInfo(HRegionInfo region) {
589     // TODO implement this.  i think this is currently broken in trunk i don't
590     //      see this getting updated.
591     //      @see HRegion.checkRegioninfoOnFilesystem()
592   }
593 
594   public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName, boolean hasMob)
595       throws IOException {
596     // archive family store files
597     Path tableDir = FSUtils.getTableDir(rootdir, region.getTable());
598     HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
599 
600     // delete the family folder
601     Path familyDir = new Path(tableDir,
602       new Path(region.getEncodedName(), Bytes.toString(familyName)));
603     if (fs.delete(familyDir, true) == false) {
604       if (fs.exists(familyDir)) {
605         throw new IOException("Could not delete family "
606             + Bytes.toString(familyName) + " from FileSystem for region "
607             + region.getRegionNameAsString() + "(" + region.getEncodedName()
608             + ")");
609       }
610     }
611 
612     // archive and delete mob files
613     if (hasMob) {
614       Path mobTableDir =
615           FSUtils.getTableDir(new Path(getRootDir(), MobConstants.MOB_DIR_NAME), region.getTable());
616       HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(region.getTable());
617       Path mobFamilyDir =
618           new Path(mobTableDir,
619               new Path(mobRegionInfo.getEncodedName(), Bytes.toString(familyName)));
620       // archive mob family store files
621       MobUtils.archiveMobStoreFiles(conf, fs, mobRegionInfo, mobFamilyDir, familyName);
622 
623       if (!fs.delete(mobFamilyDir, true)) {
624         throw new IOException("Could not delete mob store files for family "
625             + Bytes.toString(familyName) + " from FileSystem region "
626             + mobRegionInfo.getRegionNameAsString() + "(" + mobRegionInfo.getEncodedName() + ")");
627       }
628     }
629   }
630 
631   public void stop() {
632     if (splitLogManager != null) {
633       this.splitLogManager.stop();
634     }
635   }
636 
637   /**
638    * Delete column of a table
639    * @param tableName
640    * @param familyName
641    * @return Modified HTableDescriptor with requested column deleted.
642    * @throws IOException
643    */
644   public HTableDescriptor deleteColumn(TableName tableName, byte[] familyName)
645       throws IOException {
646     LOG.info("DeleteColumn. Table = " + tableName
647         + " family = " + Bytes.toString(familyName));
648     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
649     htd.removeFamily(familyName);
650     this.services.getTableDescriptors().add(htd);
651     return htd;
652   }
653 
654   /**
655    * Modify Column of a table
656    * @param tableName
657    * @param hcd HColumnDesciptor
658    * @return Modified HTableDescriptor with the column modified.
659    * @throws IOException
660    */
661   public HTableDescriptor modifyColumn(TableName tableName, HColumnDescriptor hcd)
662       throws IOException {
663     LOG.info("AddModifyColumn. Table = " + tableName
664         + " HCD = " + hcd.toString());
665 
666     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
667     byte [] familyName = hcd.getName();
668     if(!htd.hasFamily(familyName)) {
669       throw new InvalidFamilyOperationException("Family '" +
670         Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
671     }
672     htd.modifyFamily(hcd);
673     this.services.getTableDescriptors().add(htd);
674     return htd;
675   }
676 
677   /**
678    * Add column to a table
679    * @param tableName
680    * @param hcd
681    * @return Modified HTableDescriptor with new column added.
682    * @throws IOException
683    */
684   public HTableDescriptor addColumn(TableName tableName, HColumnDescriptor hcd)
685       throws IOException {
686     LOG.info("AddColumn. Table = " + tableName + " HCD = " +
687       hcd.toString());
688     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
689     if (htd == null) {
690       throw new InvalidFamilyOperationException("Family '" +
691         hcd.getNameAsString() + "' cannot be modified as HTD is null");
692     }
693     htd.addFamily(hcd);
694     this.services.getTableDescriptors().add(htd);
695     return htd;
696   }
697 
698   /**
699    * The function is used in SSH to set recovery mode based on configuration after all outstanding
700    * log split tasks drained.
701    * @throws IOException
702    */
703   public void setLogRecoveryMode() throws IOException {
704       this.splitLogManager.setRecoveryMode(false);
705   }
706 
707   public RecoveryMode getLogRecoveryMode() {
708     return this.splitLogManager.getRecoveryMode();
709   }
710 }