View Javadoc

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