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.hbase.HColumnDescriptor;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.HTableDescriptor;
40  import org.apache.hadoop.hbase.InvalidFamilyOperationException;
41  import org.apache.hadoop.hbase.RemoteExceptionHandler;
42  import org.apache.hadoop.hbase.Server;
43  import org.apache.hadoop.hbase.ServerName;
44  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
45  import org.apache.hadoop.hbase.regionserver.HRegion;
46  import org.apache.hadoop.hbase.regionserver.wal.HLog;
47  import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
48  import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
51  import org.apache.hadoop.hbase.util.FSTableDescriptors;
52  import org.apache.hadoop.hbase.util.FSUtils;
53  
54  /**
55   * This class abstracts a bunch of operations the HMaster needs to interact with
56   * the underlying file system, including splitting log files, checking file
57   * system status, etc.
58   */
59  public class MasterFileSystem {
60    private static final Log LOG = LogFactory.getLog(MasterFileSystem.class.getName());
61    // HBase configuration
62    Configuration conf;
63    // master status
64    Server master;
65    // metrics for master
66    MasterMetrics metrics;
67    // Persisted unique cluster ID
68    private String clusterId;
69    // Keep around for convenience.
70    private final FileSystem fs;
71    // Is the fileystem ok?
72    private volatile boolean fsOk = true;
73    // The Path to the old logs dir
74    private final Path oldLogDir;
75    // root hbase directory on the FS
76    private final Path rootdir;
77    // create the split log lock
78    final Lock splitLogLock = new ReentrantLock();
79    final boolean distributedLogSplitting;
80    final SplitLogManager splitLogManager;
81    private final MasterServices services;
82  
83    public MasterFileSystem(Server master, MasterServices services,
84        MasterMetrics metrics)
85    throws IOException {
86      this.conf = master.getConfiguration();
87      this.master = master;
88      this.services = services;
89      this.metrics = metrics;
90      // Set filesystem to be that of this.rootdir else we get complaints about
91      // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
92      // default localfs.  Presumption is that rootdir is fully-qualified before
93      // we get to here with appropriate fs scheme.
94      this.rootdir = FSUtils.getRootDir(conf);
95      // Cover both bases, the old way of setting default fs and the new.
96      // We're supposed to run on 0.20 and 0.21 anyways.
97      this.fs = this.rootdir.getFileSystem(conf);
98      String fsUri = this.fs.getUri().toString();
99      conf.set("fs.default.name", fsUri);
100     conf.set("fs.defaultFS", fsUri);
101     this.distributedLogSplitting =
102       conf.getBoolean("hbase.master.distributed.log.splitting", true);
103     if (this.distributedLogSplitting) {
104       this.splitLogManager = new SplitLogManager(master.getZooKeeper(),
105           master.getConfiguration(), master, master.getServerName().toString());
106       this.splitLogManager.finishInitialization();
107     } else {
108       this.splitLogManager = null;
109     }
110     // setup the filesystem variable
111     // set up the archived logs path
112     this.oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
113     createInitialFileSystemLayout();
114   }
115 
116   /**
117    * Create initial layout in filesystem.
118    * <ol>
119    * <li>Check if the root region exists and is readable, if not create it.
120    * Create hbase.version and the -ROOT- directory if not one.
121    * </li>
122    * <li>Create a log archive directory for RS to put archived logs</li>
123    * </ol>
124    * Idempotent.
125    */
126   private void createInitialFileSystemLayout() throws IOException {
127     // check if the root directory exists
128     checkRootDir(this.rootdir, conf, this.fs);
129 
130     // Make sure the region servers can archive their old logs
131     if(!this.fs.exists(this.oldLogDir)) {
132       this.fs.mkdirs(this.oldLogDir);
133     }
134   }
135 
136   public FileSystem getFileSystem() {
137     return this.fs;
138   }
139 
140   /**
141    * Get the directory where old logs go
142    * @return the dir
143    */
144   public Path getOldLogDir() {
145     return this.oldLogDir;
146   }
147 
148   /**
149    * Checks to see if the file system is still accessible.
150    * If not, sets closed
151    * @return false if file system is not available
152    */
153   public boolean checkFileSystem() {
154     if (this.fsOk) {
155       try {
156         FSUtils.checkFileSystemAvailable(this.fs);
157         FSUtils.checkDfsSafeMode(this.conf);
158       } catch (IOException e) {
159         master.abort("Shutting down HBase cluster: file system not available", e);
160         this.fsOk = false;
161       }
162     }
163     return this.fsOk;
164   }
165 
166   /**
167    * @return HBase root dir.
168    */
169   public Path getRootDir() {
170     return this.rootdir;
171   }
172 
173   /**
174    * @return The unique identifier generated for this cluster
175    */
176   public String getClusterId() {
177     return clusterId;
178   }
179 
180   /**
181    * Inspect the log directory to recover any log file without
182    * an active region server.
183    * @param onlineServers Set of online servers keyed by
184    * {@link ServerName}
185    */
186   void splitLogAfterStartup(final Set<ServerName> onlineServers) {
187     boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors",
188         HLog.SPLIT_SKIP_ERRORS_DEFAULT);
189     Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
190     do {
191       List<ServerName> serverNames = new ArrayList<ServerName>();
192       try {
193         if (!this.fs.exists(logsDirPath)) return;
194         FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
195 
196         if (logFolders == null || logFolders.length == 0) {
197           LOG.debug("No log files to split, proceeding...");
198           return;
199         }
200         for (FileStatus status : logFolders) {
201           String sn = status.getPath().getName();
202           // truncate splitting suffix if present (for ServerName parsing)
203           if (sn.endsWith(HLog.SPLITTING_EXT)) {
204             sn = sn.substring(0, sn.length() - HLog.SPLITTING_EXT.length());
205           }
206           ServerName serverName = ServerName.parseServerName(sn);
207           if (!onlineServers.contains(serverName)) {
208             LOG.info("Log folder " + status.getPath() + " doesn't belong "
209                 + "to a known region server, splitting");
210             serverNames.add(serverName);
211           } else {
212             LOG.info("Log folder " + status.getPath()
213                 + " belongs to an existing region server");
214           }
215         }
216         splitLog(serverNames);
217         retrySplitting = false;
218       } catch (IOException ioe) {
219         LOG.warn("Failed splitting of " + serverNames, ioe);
220         if (!checkFileSystem()) {
221           LOG.warn("Bad Filesystem, exiting");
222           Runtime.getRuntime().halt(1);
223         }
224         try {
225           if (retrySplitting) {
226             Thread.sleep(conf.getInt(
227               "hbase.hlog.split.failure.retry.interval", 30 * 1000));
228           }
229         } catch (InterruptedException e) {
230           LOG.warn("Interrupted, returning w/o splitting at startup");
231           Thread.currentThread().interrupt();
232           retrySplitting = false;
233         }
234       }
235     } while (retrySplitting);
236   }
237   
238   public void splitLog(final ServerName serverName) throws IOException {
239     List<ServerName> serverNames = new ArrayList<ServerName>();
240     serverNames.add(serverName);
241     splitLog(serverNames);
242   }
243   
244   public void splitLog(final List<ServerName> serverNames) throws IOException {
245     long splitTime = 0, splitLogSize = 0;
246     List<Path> logDirs = new ArrayList<Path>();
247     for(ServerName serverName: serverNames){
248       Path logDir = new Path(this.rootdir,
249         HLog.getHLogDirectoryName(serverName.toString()));
250       Path splitDir = logDir.suffix(HLog.SPLITTING_EXT);
251       // rename the directory so a rogue RS doesn't create more HLogs
252       if (fs.exists(logDir)) {
253         if (!this.fs.rename(logDir, splitDir)) {
254           throw new IOException("Failed fs.rename for log split: " + logDir);
255         }
256         logDir = splitDir;
257         LOG.debug("Renamed region directory: " + splitDir);
258       } else if (!fs.exists(splitDir)) {
259         LOG.info("Log dir for server " + serverName + " does not exist");
260         continue;
261       }
262       logDirs.add(splitDir);
263     }
264 
265     if (logDirs.isEmpty()) {
266       LOG.info("No logs to split");
267       return;
268     }
269       
270     if (distributedLogSplitting) {
271       for (ServerName serverName : serverNames) {
272         splitLogManager.handleDeadWorker(serverName.toString());
273       }
274       splitTime = EnvironmentEdgeManager.currentTimeMillis();
275       try {
276         splitLogSize = splitLogManager.splitLogDistributed(logDirs);
277       } catch (OrphanHLogAfterSplitException e) {
278         LOG.warn("Retrying distributed splitting for " + serverNames, e);
279         splitLogManager.splitLogDistributed(logDirs);
280       }
281       splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
282     } else {
283       for(Path logDir: logDirs){
284         // splitLogLock ensures that dead region servers' logs are processed
285         // one at a time
286         this.splitLogLock.lock();
287         try {              
288           HLogSplitter splitter = HLogSplitter.createLogSplitter(
289             conf, rootdir, logDir, oldLogDir, this.fs);
290           try {
291             // If FS is in safe mode, just wait till out of it.
292             FSUtils.waitOnSafeMode(conf, conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 1000));
293             splitter.splitLog();
294           } catch (OrphanHLogAfterSplitException e) {
295             LOG.warn("Retrying splitting because of:", e);
296             //An HLogSplitter instance can only be used once.  Get new instance.
297             splitter = HLogSplitter.createLogSplitter(conf, rootdir, logDir,
298               oldLogDir, this.fs);
299             splitter.splitLog();
300           }
301           splitTime = splitter.getTime();
302           splitLogSize = splitter.getSize();
303         } finally {
304           this.splitLogLock.unlock();
305         }
306       }
307     }
308 
309     if (this.metrics != null) {
310       this.metrics.addSplit(splitTime, splitLogSize);
311     }
312   }
313 
314   /**
315    * Get the rootdir.  Make sure its wholesome and exists before returning.
316    * @param rd
317    * @param conf
318    * @param fs
319    * @return hbase.rootdir (after checks for existence and bootstrapping if
320    * needed populating the directory with necessary bootup files).
321    * @throws IOException
322    */
323   private Path checkRootDir(final Path rd, final Configuration c,
324     final FileSystem fs)
325   throws IOException {
326     // If FS is in safe mode wait till out of it.
327     FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
328         10 * 1000));
329     // Filesystem is good. Go ahead and check for hbase.rootdir.
330     if (!fs.exists(rd)) {
331       fs.mkdirs(rd);
332       // DFS leaves safe mode with 0 DNs when there are 0 blocks.
333       // We used to handle this by checking the current DN count and waiting until
334       // it is nonzero. With security, the check for datanode count doesn't work --
335       // it is a privileged op. So instead we adopt the strategy of the jobtracker
336       // and simply retry file creation during bootstrap indefinitely. As soon as
337       // there is one datanode it will succeed. Permission problems should have
338       // already been caught by mkdirs above.
339       FSUtils.setVersion(fs, rd, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
340         10 * 1000));
341     } else {
342       // as above
343       FSUtils.checkVersion(fs, rd, true, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
344         10 * 1000));
345     }
346     // Make sure cluster ID exists
347     if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
348         HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
349       FSUtils.setClusterId(fs, rd, UUID.randomUUID().toString(), c.getInt(
350           HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
351     }
352     clusterId = FSUtils.getClusterId(fs, rd);
353 
354     // Make sure the root region directory exists!
355     if (!FSUtils.rootRegionExists(fs, rd)) {
356       bootstrap(rd, c);
357     }
358     createRootTableInfo(rd);
359     return rd;
360   }
361 
362   private void createRootTableInfo(Path rd) throws IOException {
363     // Create ROOT tableInfo if required.
364     if (!FSTableDescriptors.isTableInfoExists(fs, rd,
365         Bytes.toString(HRegionInfo.ROOT_REGIONINFO.getTableName()))) {
366       FSTableDescriptors.createTableDescriptor(HTableDescriptor.ROOT_TABLEDESC, this.conf);
367     }
368   }
369 
370   private static void bootstrap(final Path rd, final Configuration c)
371   throws IOException {
372     LOG.info("BOOTSTRAP: creating ROOT and first META regions");
373     try {
374       // Bootstrapping, make sure blockcache is off.  Else, one will be
375       // created here in bootstap and it'll need to be cleaned up.  Better to
376       // not make it in first place.  Turn off block caching for bootstrap.
377       // Enable after.
378       HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
379       setInfoFamilyCachingForRoot(false);
380       HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
381       setInfoFamilyCachingForMeta(false);
382       HRegion root = HRegion.createHRegion(rootHRI, rd, c,
383           HTableDescriptor.ROOT_TABLEDESC);
384       HRegion meta = HRegion.createHRegion(metaHRI, rd, c,
385           HTableDescriptor.META_TABLEDESC);
386       setInfoFamilyCachingForRoot(true);
387       setInfoFamilyCachingForMeta(true);
388       // Add first region from the META table to the ROOT region.
389       HRegion.addRegionToMETA(root, meta);
390       root.close();
391       root.getLog().closeAndDelete();
392       meta.close();
393       meta.getLog().closeAndDelete();
394     } catch (IOException e) {
395       e = RemoteExceptionHandler.checkIOException(e);
396       LOG.error("bootstrap", e);
397       throw e;
398     }
399   }
400 
401   /**
402    * Enable in-memory caching for -ROOT-
403    */
404   public static void setInfoFamilyCachingForRoot(final boolean b) {
405     for (HColumnDescriptor hcd:
406         HTableDescriptor.ROOT_TABLEDESC.getColumnFamilies()) {
407        if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
408          hcd.setBlockCacheEnabled(b);
409          hcd.setInMemory(b);
410      }
411     }
412   }
413 
414   /**
415    * Enable in memory caching for .META.
416    */
417   public static void setInfoFamilyCachingForMeta(final boolean b) {
418     for (HColumnDescriptor hcd:
419         HTableDescriptor.META_TABLEDESC.getColumnFamilies()) {
420       if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
421         hcd.setBlockCacheEnabled(b);
422         hcd.setInMemory(b);
423       }
424     }
425   }
426 
427 
428   public void deleteRegion(HRegionInfo region) throws IOException {
429     fs.delete(HRegion.getRegionDir(rootdir, region), true);
430   }
431 
432   public void deleteTable(byte[] tableName) throws IOException {
433     fs.delete(new Path(rootdir, Bytes.toString(tableName)), true);
434   }
435 
436   public void updateRegionInfo(HRegionInfo region) {
437     // TODO implement this.  i think this is currently broken in trunk i don't
438     //      see this getting updated.
439     //      @see HRegion.checkRegioninfoOnFilesystem()
440   }
441 
442   public void stop() {
443     if (splitLogManager != null) {
444       this.splitLogManager.stop();
445     }
446   }
447 
448   /**
449    * Create new HTableDescriptor in HDFS.
450    * 
451    * @param htableDescriptor
452    */
453   public void createTableDescriptor(HTableDescriptor htableDescriptor)
454       throws IOException {
455     FSTableDescriptors.createTableDescriptor(htableDescriptor, conf);
456   }
457 
458   /**
459    * Delete column of a table
460    * @param tableName
461    * @param familyName
462    * @return Modified HTableDescriptor with requested column deleted.
463    * @throws IOException
464    */
465   public HTableDescriptor deleteColumn(byte[] tableName, byte[] familyName)
466       throws IOException {
467     LOG.info("DeleteColumn. Table = " + Bytes.toString(tableName)
468         + " family = " + Bytes.toString(familyName));
469     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
470     htd.removeFamily(familyName);
471     this.services.getTableDescriptors().add(htd);
472     return htd;
473   }
474 
475   /**
476    * Modify Column of a table
477    * @param tableName
478    * @param hcd HColumnDesciptor
479    * @return Modified HTableDescriptor with the column modified.
480    * @throws IOException
481    */
482   public HTableDescriptor modifyColumn(byte[] tableName, HColumnDescriptor hcd)
483       throws IOException {
484     LOG.info("AddModifyColumn. Table = " + Bytes.toString(tableName)
485         + " HCD = " + hcd.toString());
486 
487     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
488     byte [] familyName = hcd.getName();
489     if(!htd.hasFamily(familyName)) {
490       throw new InvalidFamilyOperationException("Family '" +
491         Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
492     }
493     htd.addFamily(hcd);
494     this.services.getTableDescriptors().add(htd);
495     return htd;
496   }
497 
498   /**
499    * Add column to a table
500    * @param tableName
501    * @param hcd
502    * @return Modified HTableDescriptor with new column added.
503    * @throws IOException
504    */
505   public HTableDescriptor addColumn(byte[] tableName, HColumnDescriptor hcd)
506       throws IOException {
507     LOG.info("AddColumn. Table = " + Bytes.toString(tableName) + " HCD = " +
508       hcd.toString());
509     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
510     if (htd == null) {
511       throw new InvalidFamilyOperationException("Family '" +
512         hcd.getNameAsString() + "' cannot be modified as HTD is null");
513     }
514     htd.addFamily(hcd);
515     this.services.getTableDescriptors().add(htd);
516     return htd;
517   }
518 }