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 when splitting log files e.g.
049 * 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  // old WALs directory size in bytes
093  private long oldWALsDirSize;
094
095  /**
096   * Superceded by {@link SplitWALManager}; i.e. procedure-based WAL splitting rather than 'classic'
097   * zk-coordinated WAL splitting.
098   * @deprecated since 2.3.0 and 3.0.0 to be removed in 4.0.0; replaced by {@link SplitWALManager}.
099   * @see SplitWALManager
100   */
101  @Deprecated
102  private final SplitLogManager splitLogManager;
103
104  // Is the fileystem ok?
105  private volatile boolean fsOk = true;
106
107  public MasterWalManager(MasterServices services) throws IOException {
108    this(services.getConfiguration(), services.getMasterFileSystem().getWALFileSystem(), services);
109  }
110
111  public MasterWalManager(Configuration conf, FileSystem fs, MasterServices services)
112    throws IOException {
113    this.fs = fs;
114    this.conf = conf;
115    this.rootDir = CommonFSUtils.getWALRootDir(conf);
116    this.services = services;
117    this.splitLogManager = new SplitLogManager(services, conf);
118    this.oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
119    this.oldWALsDirSize = 0;
120  }
121
122  public void stop() {
123    if (splitLogManager != null) {
124      splitLogManager.stop();
125    }
126  }
127
128  SplitLogManager getSplitLogManager() {
129    return this.splitLogManager;
130  }
131
132  /**
133   * Get the directory where old logs go
134   * @return the dir
135   */
136  Path getOldLogDir() {
137    return this.oldLogDir;
138  }
139
140  public void updateOldWALsDirSize() throws IOException {
141    this.oldWALsDirSize = fs.getContentSummary(this.oldLogDir).getLength();
142  }
143
144  public long getOldWALsDirSize() {
145    return this.oldWALsDirSize;
146  }
147
148  public FileSystem getFileSystem() {
149    return this.fs;
150  }
151
152  /**
153   * Checks to see if the file system is still accessible. If not, sets closed
154   * @return false if file system is not available
155   */
156  private boolean checkFileSystem() {
157    if (this.fsOk) {
158      try {
159        FSUtils.checkFileSystemAvailable(this.fs);
160        FSUtils.checkDfsSafeMode(this.conf);
161      } catch (IOException e) {
162        services.abort("Shutting down HBase cluster: file system not available", e);
163        this.fsOk = false;
164      }
165    }
166    return this.fsOk;
167  }
168
169  /**
170   * Get Servernames which are currently splitting; paths have a '-splitting' suffix.
171   */
172  public Set<ServerName> getSplittingServersFromWALDir() throws IOException {
173    return getServerNamesFromWALDirPath(
174      p -> p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
175  }
176
177  /**
178   * Get Servernames that COULD BE 'alive'; excludes those that have a '-splitting' suffix as these
179   * are already being split -- they cannot be 'alive'.
180   */
181  public Set<ServerName> getLiveServersFromWALDir() throws IOException {
182    return getServerNamesFromWALDirPath(
183      p -> !p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
184  }
185
186  /** Returns listing of ServerNames found by parsing WAL directory paths in FS. */
187  public Set<ServerName> getServerNamesFromWALDirPath(final PathFilter filter) throws IOException {
188    FileStatus[] walDirForServerNames = getWALDirPaths(filter);
189    return Stream.of(walDirForServerNames).map(s -> {
190      ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(s.getPath());
191      if (serverName == null) {
192        LOG.warn("Log folder {} doesn't look like its name includes a "
193          + "region server name; leaving in place. If you see later errors about missing "
194          + "write ahead logs they may be saved in this location.", s.getPath());
195        return null;
196      }
197      return serverName;
198    }).filter(s -> s != null).collect(Collectors.toSet());
199  }
200
201  /**
202   * Returns List of all RegionServer WAL dirs; i.e. this.rootDir/HConstants.HREGION_LOGDIR_NAME.
203   */
204  public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
205    Path walDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
206    FileStatus[] walDirForServerNames = CommonFSUtils.listStatus(fs, walDirPath, filter);
207    return walDirForServerNames == null ? new FileStatus[0] : walDirForServerNames;
208  }
209
210  /**
211   * Inspect the log directory to find dead servers which need recovery work
212   * @return A set of ServerNames which aren't running but still have WAL files left in file system
213   * @deprecated With proc-v2, we can record the crash server with procedure store, so do not need
214   *             to scan the wal directory to find out the splitting wal directory any more. Leave
215   *             it here only because {@code RecoverMetaProcedure}(which is also deprecated) uses
216   *             it.
217   */
218  @Deprecated
219  public Set<ServerName> getFailedServersFromLogFolders() throws IOException {
220    boolean retrySplitting =
221      !conf.getBoolean(WALSplitter.SPLIT_SKIP_ERRORS_KEY, WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
222
223    Set<ServerName> serverNames = new HashSet<>();
224    Path logsDirPath = new Path(this.rootDir, HConstants.HREGION_LOGDIR_NAME);
225
226    do {
227      if (services.isStopped()) {
228        LOG.warn("Master stopped while trying to get failed servers.");
229        break;
230      }
231      try {
232        if (!this.fs.exists(logsDirPath)) return serverNames;
233        FileStatus[] logFolders = CommonFSUtils.listStatus(this.fs, logsDirPath, null);
234        // Get online servers after getting log folders to avoid log folder deletion of newly
235        // checked in region servers . see HBASE-5916
236        Set<ServerName> onlineServers = services.getServerManager().getOnlineServers().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 = CommonFSUtils.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 =
249            AbstractFSWALProvider.getServerNameFromWALDirectoryName(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    splitLog(Collections.<ServerName> singleton(serverName));
287  }
288
289  /**
290   * Specialized method to handle the splitting for meta WAL
291   * @param serverName logs belonging to this server will be split
292   */
293  public void splitMetaLog(final ServerName serverName) throws IOException {
294    splitMetaLog(Collections.<ServerName> singleton(serverName));
295  }
296
297  /**
298   * Specialized method to handle the splitting for meta WAL
299   * @param serverNames logs belonging to these servers will be split
300   */
301  public void splitMetaLog(final Set<ServerName> serverNames) throws IOException {
302    splitLog(serverNames, META_FILTER);
303  }
304
305  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UL_UNRELEASED_LOCK",
306      justification = "We only release this lock when we set it. Updates to code "
307        + "that uses it should verify use of the guard boolean.")
308  List<Path> getLogDirs(final Set<ServerName> serverNames) throws IOException {
309    List<Path> logDirs = new ArrayList<>();
310    boolean needReleaseLock = false;
311    if (!this.services.isInitialized()) {
312      // during master initialization, we could have multiple places splitting a same wal
313      // XXX: Does this still exist after we move to proc-v2?
314      this.splitLogLock.lock();
315      needReleaseLock = true;
316    }
317    try {
318      for (ServerName serverName : serverNames) {
319        Path logDir =
320          new Path(this.rootDir, AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
321        Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
322        // Rename the directory so a rogue RS doesn't create more WALs
323        if (fs.exists(logDir)) {
324          if (!this.fs.rename(logDir, splitDir)) {
325            throw new IOException("Failed fs.rename for log split: " + logDir);
326          }
327          logDir = splitDir;
328          LOG.debug("Renamed region directory: " + splitDir);
329        } else if (!fs.exists(splitDir)) {
330          LOG.info("Log dir for server " + serverName + " does not exist");
331          continue;
332        }
333        logDirs.add(splitDir);
334      }
335    } catch (IOException ioe) {
336      if (!checkFileSystem()) {
337        this.services.abort("Aborting due to filesystem unavailable", ioe);
338        throw ioe;
339      }
340    } finally {
341      if (needReleaseLock) {
342        this.splitLogLock.unlock();
343      }
344    }
345    return logDirs;
346  }
347
348  public void splitLog(final Set<ServerName> serverNames) throws IOException {
349    splitLog(serverNames, NON_META_FILTER);
350  }
351
352  /**
353   * This method is the base split method that splits WAL files matching a filter. Callers should
354   * pass the appropriate filter for meta and non-meta WALs.
355   * @param serverNames logs belonging to these servers will be split; this will rename the log
356   *                    directory out from under a soft-failed server
357   */
358  public void splitLog(final Set<ServerName> serverNames, PathFilter filter) throws IOException {
359    long splitTime = 0, splitLogSize = 0;
360    List<Path> logDirs = getLogDirs(serverNames);
361
362    splitLogManager.handleDeadWorkers(serverNames);
363    splitTime = EnvironmentEdgeManager.currentTime();
364    splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter);
365    splitTime = EnvironmentEdgeManager.currentTime() - splitTime;
366
367    if (this.metricsMasterFilesystem != null) {
368      if (filter == META_FILTER) {
369        this.metricsMasterFilesystem.addMetaWALSplit(splitTime, splitLogSize);
370      } else {
371        this.metricsMasterFilesystem.addSplit(splitTime, splitLogSize);
372      }
373    }
374  }
375
376  /**
377   * The hbase:meta region may OPEN and CLOSE without issue on a server and then move elsewhere. On
378   * CLOSE, the WAL for the hbase:meta table may not be archived yet (The WAL is only needed if
379   * hbase:meta did not close cleanaly). Since meta region is no long on this server, the
380   * ServerCrashProcedure won't split these leftover hbase:meta WALs, just leaving them in the WAL
381   * splitting dir. If we try to delete the WAL splitting for the server, it fail since the dir is
382   * not totally empty. We can safely archive these hbase:meta log; then the WAL dir can be deleted.
383   * @param serverName the server to archive meta log
384   */
385  public void archiveMetaLog(final ServerName serverName) {
386    try {
387      Path logDir =
388        new Path(this.rootDir, AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
389      Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
390      if (fs.exists(splitDir)) {
391        FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
392        if (logfiles != null) {
393          for (FileStatus status : logfiles) {
394            if (!status.isDir()) {
395              Path newPath = AbstractFSWAL.getWALArchivePath(this.oldLogDir, status.getPath());
396              if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
397                LOG.warn("Unable to move  " + status.getPath() + " to " + newPath);
398              } else {
399                LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);
400              }
401            }
402          }
403        }
404        if (!fs.delete(splitDir, false)) {
405          LOG.warn("Unable to delete log dir. Ignoring. " + splitDir);
406        }
407      }
408    } catch (IOException ie) {
409      LOG.warn("Failed archiving meta log for server " + serverName, ie);
410    }
411  }
412}