001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.master;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Collections;
023import java.util.HashSet;
024import java.util.List;
025import java.util.Set;
026import java.util.concurrent.locks.Lock;
027import java.util.concurrent.locks.ReentrantLock;
028import java.util.stream.Collectors;
029import java.util.stream.Stream;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.fs.FileStatus;
032import org.apache.hadoop.fs.FileSystem;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.fs.PathFilter;
035import org.apache.hadoop.hbase.HConstants;
036import org.apache.hadoop.hbase.ServerName;
037import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
038import org.apache.hadoop.hbase.util.CommonFSUtils;
039import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
040import org.apache.hadoop.hbase.util.FSUtils;
041import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
042import org.apache.hadoop.hbase.wal.WALSplitter;
043import org.apache.yetus.audience.InterfaceAudience;
044import org.slf4j.Logger;
045import org.slf4j.LoggerFactory;
046
047/**
048 * This class abstracts a bunch of operations the HMaster needs
049 * when splitting log files e.g. finding log files, dirs etc.
050 */
051@InterfaceAudience.Private
052public class MasterWalManager {
053  private static final Logger LOG = LoggerFactory.getLogger(MasterWalManager.class);
054
055  /**
056   * Filter *in* WAL files that are for the hbase:meta Region.
057   */
058  final static PathFilter META_FILTER = new PathFilter() {
059    @Override
060    public boolean accept(Path p) {
061      return AbstractFSWALProvider.isMetaFile(p);
062    }
063  };
064
065  /**
066   * Filter *out* WAL files that are for the hbase:meta Region; i.e. return user-space WALs only.
067   */
068  public final static PathFilter NON_META_FILTER = new PathFilter() {
069    @Override
070    public boolean accept(Path p) {
071      return !AbstractFSWALProvider.isMetaFile(p);
072    }
073  };
074
075  // metrics for master
076  // TODO: Rename it, since those metrics are split-manager related
077  private final MetricsMasterFileSystem metricsMasterFilesystem = new MetricsMasterFileSystem();
078
079  // Keep around for convenience.
080  private final MasterServices services;
081  private final Configuration conf;
082  private final FileSystem fs;
083
084  // The Path to the old logs dir
085  private final Path oldLogDir;
086
087  private final Path rootDir;
088
089  // create the split log lock
090  private final Lock splitLogLock = new ReentrantLock();
091
092  /**
093   * Superceded by {@link SplitWALManager}; i.e. procedure-based WAL splitting rather than
094   *   'classic' zk-coordinated WAL splitting.
095   * @deprecated  since 2.3.0 and 3.0.0 to be removed in 4.0.0; replaced by {@link SplitWALManager}.
096   * @see SplitWALManager
097   */
098  @Deprecated
099  private final SplitLogManager splitLogManager;
100
101  // Is the fileystem ok?
102  private volatile boolean fsOk = true;
103
104  public MasterWalManager(MasterServices services) throws IOException {
105    this(services.getConfiguration(), services.getMasterFileSystem().getWALFileSystem(), services);
106  }
107
108  public MasterWalManager(Configuration conf, FileSystem fs,  MasterServices services)
109      throws IOException {
110    this.fs = fs;
111    this.conf = conf;
112    this.rootDir = CommonFSUtils.getWALRootDir(conf);
113    this.services = services;
114    this.splitLogManager = new SplitLogManager(services, conf);
115    this.oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
116  }
117
118  public void stop() {
119    if (splitLogManager != null) {
120      splitLogManager.stop();
121    }
122  }
123
124  SplitLogManager getSplitLogManager() {
125    return this.splitLogManager;
126  }
127
128  /**
129   * Get the directory where old logs go
130   * @return the dir
131   */
132  Path getOldLogDir() {
133    return this.oldLogDir;
134  }
135
136  public FileSystem getFileSystem() {
137    return this.fs;
138  }
139
140  /**
141   * Checks to see if the file system is still accessible.
142   * If not, sets closed
143   * @return false if file system is not available
144   */
145  private boolean checkFileSystem() {
146    if (this.fsOk) {
147      try {
148        FSUtils.checkFileSystemAvailable(this.fs);
149        FSUtils.checkDfsSafeMode(this.conf);
150      } catch (IOException e) {
151        services.abort("Shutting down HBase cluster: file system not available", e);
152        this.fsOk = false;
153      }
154    }
155    return this.fsOk;
156  }
157
158  /**
159   * Get Servernames which are currently splitting; paths have a '-splitting' suffix.
160   * @return ServerName
161   * @throws IOException IOException
162   */
163  public Set<ServerName> getSplittingServersFromWALDir() throws  IOException {
164    return getServerNamesFromWALDirPath(
165      p -> p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
166  }
167
168  /**
169   * Get Servernames that COULD BE 'alive'; excludes those that have a '-splitting' suffix as these
170   * are already being split -- they cannot be 'alive'.
171   * @return ServerName
172   * @throws IOException IOException
173   */
174  public Set<ServerName> getLiveServersFromWALDir() throws IOException {
175    return getServerNamesFromWALDirPath(
176      p -> !p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
177  }
178
179  /**
180   * @return listing of ServerNames found by parsing WAL directory paths in FS.
181   */
182  public Set<ServerName> getServerNamesFromWALDirPath(final PathFilter filter) throws IOException {
183    FileStatus[] walDirForServerNames = getWALDirPaths(filter);
184    return Stream.of(walDirForServerNames).map(s -> {
185      ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(s.getPath());
186      if (serverName == null) {
187        LOG.warn("Log folder {} doesn't look like its name includes a " +
188          "region server name; leaving in place. If you see later errors about missing " +
189          "write ahead logs they may be saved in this location.", s.getPath());
190        return null;
191      }
192      return serverName;
193    }).filter(s -> s != null).collect(Collectors.toSet());
194  }
195
196  /**
197   * @return List of all RegionServer WAL dirs; i.e. this.rootDir/HConstants.HREGION_LOGDIR_NAME.
198   */
199  public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
200    Path walDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
201    FileStatus[] walDirForServerNames = CommonFSUtils.listStatus(fs, walDirPath, filter);
202    return walDirForServerNames == null? new FileStatus[0]: walDirForServerNames;
203  }
204
205  /**
206   * Inspect the log directory to find dead servers which need recovery work
207   * @return A set of ServerNames which aren't running but still have WAL files left in file system
208   * @deprecated With proc-v2, we can record the crash server with procedure store, so do not need
209   *             to scan the wal directory to find out the splitting wal directory any more. Leave
210   *             it here only because {@code RecoverMetaProcedure}(which is also deprecated) uses
211   *             it.
212   */
213  @Deprecated
214  public Set<ServerName> getFailedServersFromLogFolders() throws IOException {
215    boolean retrySplitting = !conf.getBoolean(WALSplitter.SPLIT_SKIP_ERRORS_KEY,
216        WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
217
218    Set<ServerName> serverNames = new HashSet<>();
219    Path logsDirPath = new Path(this.rootDir, HConstants.HREGION_LOGDIR_NAME);
220
221    do {
222      if (services.isStopped()) {
223        LOG.warn("Master stopped while trying to get failed servers.");
224        break;
225      }
226      try {
227        if (!this.fs.exists(logsDirPath)) return serverNames;
228        FileStatus[] logFolders = CommonFSUtils.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 = services.getServerManager().getOnlineServers().keySet();
232
233        if (logFolders == null || logFolders.length == 0) {
234          LOG.debug("No log files to split, proceeding...");
235          return serverNames;
236        }
237        for (FileStatus status : logFolders) {
238          FileStatus[] curLogFiles = CommonFSUtils.listStatus(this.fs, status.getPath(), null);
239          if (curLogFiles == null || curLogFiles.length == 0) {
240            // Empty log folder. No recovery needed
241            continue;
242          }
243          final ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(
244              status.getPath());
245          if (null == serverName) {
246            LOG.warn("Log folder " + status.getPath() + " doesn't look like its name includes a " +
247                "region server name; leaving in place. If you see later errors about missing " +
248                "write ahead logs they may be saved in this location.");
249          } else if (!onlineServers.contains(serverName)) {
250            LOG.info("Log folder " + status.getPath() + " doesn't belong "
251                + "to a known region server, splitting");
252            serverNames.add(serverName);
253          } else {
254            LOG.info("Log folder " + status.getPath() + " belongs to an existing region server");
255          }
256        }
257        retrySplitting = false;
258      } catch (IOException ioe) {
259        LOG.warn("Failed getting failed servers to be recovered.", ioe);
260        if (!checkFileSystem()) {
261          LOG.warn("Bad Filesystem, exiting");
262          Runtime.getRuntime().halt(1);
263        }
264        try {
265          if (retrySplitting) {
266            Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000));
267          }
268        } catch (InterruptedException e) {
269          LOG.warn("Interrupted, aborting since cannot return w/o splitting");
270          Thread.currentThread().interrupt();
271          retrySplitting = false;
272          Runtime.getRuntime().halt(1);
273        }
274      }
275    } while (retrySplitting);
276
277    return serverNames;
278  }
279
280  public void splitLog(final ServerName serverName) throws IOException {
281    splitLog(Collections.<ServerName>singleton(serverName));
282  }
283
284  /**
285   * Specialized method to handle the splitting for meta WAL
286   * @param serverName logs belonging to this server will be split
287   */
288  public void splitMetaLog(final ServerName serverName) throws IOException {
289    splitMetaLog(Collections.<ServerName>singleton(serverName));
290  }
291
292  /**
293   * Specialized method to handle the splitting for meta WAL
294   * @param serverNames logs belonging to these servers will be split
295   */
296  public void splitMetaLog(final Set<ServerName> serverNames) throws IOException {
297    splitLog(serverNames, META_FILTER);
298  }
299
300  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification=
301      "We only release this lock when we set it. Updates to code that uses it should verify use " +
302      "of the guard boolean.")
303  List<Path> getLogDirs(final Set<ServerName> serverNames) throws IOException {
304    List<Path> logDirs = new ArrayList<>();
305    boolean needReleaseLock = false;
306    if (!this.services.isInitialized()) {
307      // during master initialization, we could have multiple places splitting a same wal
308      // XXX: Does this still exist after we move to proc-v2?
309      this.splitLogLock.lock();
310      needReleaseLock = true;
311    }
312    try {
313      for (ServerName serverName : serverNames) {
314        Path logDir = new Path(this.rootDir,
315          AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
316        Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
317        // Rename the directory so a rogue RS doesn't create more WALs
318        if (fs.exists(logDir)) {
319          if (!this.fs.rename(logDir, splitDir)) {
320            throw new IOException("Failed fs.rename for log split: " + logDir);
321          }
322          logDir = splitDir;
323          LOG.debug("Renamed region directory: " + splitDir);
324        } else if (!fs.exists(splitDir)) {
325          LOG.info("Log dir for server " + serverName + " does not exist");
326          continue;
327        }
328        logDirs.add(splitDir);
329      }
330    } catch (IOException ioe) {
331      if (!checkFileSystem()) {
332        this.services.abort("Aborting due to filesystem unavailable", ioe);
333        throw ioe;
334      }
335    } finally {
336      if (needReleaseLock) {
337        this.splitLogLock.unlock();
338      }
339    }
340    return logDirs;
341  }
342
343  public void splitLog(final Set<ServerName> serverNames) throws IOException {
344    splitLog(serverNames, NON_META_FILTER);
345  }
346
347  /**
348   * This method is the base split method that splits WAL files matching a filter. Callers should
349   * pass the appropriate filter for meta and non-meta WALs.
350   * @param serverNames logs belonging to these servers will be split; this will rename the log
351   *                    directory out from under a soft-failed server
352   */
353  public void splitLog(final Set<ServerName> serverNames, PathFilter filter) throws IOException {
354    long splitTime = 0, splitLogSize = 0;
355    List<Path> logDirs = getLogDirs(serverNames);
356
357    splitLogManager.handleDeadWorkers(serverNames);
358    splitTime = EnvironmentEdgeManager.currentTime();
359    splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter);
360    splitTime = EnvironmentEdgeManager.currentTime() - splitTime;
361
362    if (this.metricsMasterFilesystem != null) {
363      if (filter == META_FILTER) {
364        this.metricsMasterFilesystem.addMetaWALSplit(splitTime, splitLogSize);
365      } else {
366        this.metricsMasterFilesystem.addSplit(splitTime, splitLogSize);
367      }
368    }
369  }
370
371  /**
372   * The hbase:meta region may OPEN and CLOSE without issue on a server and then move elsewhere.
373   * On CLOSE, the WAL for the hbase:meta table may not be archived yet (The WAL is only needed if
374   * hbase:meta did not close cleanaly). Since meta region is no long on this server,
375   * the ServerCrashProcedure won't split these leftover hbase:meta WALs, just leaving them in
376   * the WAL splitting dir. If we try to delete the WAL splitting for the server,  it fail since
377   * the dir is not totally empty. We can safely archive these hbase:meta log; then the
378   * WAL dir can be deleted.
379   * @param serverName the server to archive meta log
380   */
381  public void archiveMetaLog(final ServerName serverName) {
382    try {
383      Path logDir = new Path(this.rootDir,
384          AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
385      Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
386      if (fs.exists(splitDir)) {
387        FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
388        if (logfiles != null) {
389          for (FileStatus status : logfiles) {
390            if (!status.isDir()) {
391              Path newPath = AbstractFSWAL.getWALArchivePath(this.oldLogDir,
392                  status.getPath());
393              if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
394                LOG.warn("Unable to move  " + status.getPath() + " to " + newPath);
395              } else {
396                LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);
397              }
398            }
399          }
400        }
401        if (!fs.delete(splitDir, false)) {
402          LOG.warn("Unable to delete log dir. Ignoring. " + splitDir);
403        }
404      }
405    } catch (IOException ie) {
406      LOG.warn("Failed archiving meta log for server " + serverName, ie);
407    }
408  }
409}