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  /**
093   * Superceded by {@link SplitWALManager}; i.e. procedure-based WAL splitting rather than 'classic'
094   * 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. If not, sets closed
142   * @return false if file system is not available
143   */
144  private boolean checkFileSystem() {
145    if (this.fsOk) {
146      try {
147        FSUtils.checkFileSystemAvailable(this.fs);
148        FSUtils.checkDfsSafeMode(this.conf);
149      } catch (IOException e) {
150        services.abort("Shutting down HBase cluster: file system not available", e);
151        this.fsOk = false;
152      }
153    }
154    return this.fsOk;
155  }
156
157  /**
158   * Get Servernames which are currently splitting; paths have a '-splitting' suffix.
159   */
160  public Set<ServerName> getSplittingServersFromWALDir() throws IOException {
161    return getServerNamesFromWALDirPath(
162      p -> p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
163  }
164
165  /**
166   * Get Servernames that COULD BE 'alive'; excludes those that have a '-splitting' suffix as these
167   * are already being split -- they cannot be 'alive'.
168   */
169  public Set<ServerName> getLiveServersFromWALDir() throws IOException {
170    return getServerNamesFromWALDirPath(
171      p -> !p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
172  }
173
174  /** Returns listing of ServerNames found by parsing WAL directory paths in FS. */
175  public Set<ServerName> getServerNamesFromWALDirPath(final PathFilter filter) throws IOException {
176    FileStatus[] walDirForServerNames = getWALDirPaths(filter);
177    return Stream.of(walDirForServerNames).map(s -> {
178      ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(s.getPath());
179      if (serverName == null) {
180        LOG.warn("Log folder {} doesn't look like its name includes a "
181          + "region server name; leaving in place. If you see later errors about missing "
182          + "write ahead logs they may be saved in this location.", s.getPath());
183        return null;
184      }
185      return serverName;
186    }).filter(s -> s != null).collect(Collectors.toSet());
187  }
188
189  /**
190   * Returns List of all RegionServer WAL dirs; i.e. this.rootDir/HConstants.HREGION_LOGDIR_NAME.
191   */
192  public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
193    Path walDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
194    FileStatus[] walDirForServerNames = CommonFSUtils.listStatus(fs, walDirPath, filter);
195    return walDirForServerNames == null ? new FileStatus[0] : walDirForServerNames;
196  }
197
198  /**
199   * Inspect the log directory to find dead servers which need recovery work
200   * @return A set of ServerNames which aren't running but still have WAL files left in file system
201   * @deprecated With proc-v2, we can record the crash server with procedure store, so do not need
202   *             to scan the wal directory to find out the splitting wal directory any more. Leave
203   *             it here only because {@code RecoverMetaProcedure}(which is also deprecated) uses
204   *             it.
205   */
206  @Deprecated
207  public Set<ServerName> getFailedServersFromLogFolders() throws IOException {
208    boolean retrySplitting =
209      !conf.getBoolean(WALSplitter.SPLIT_SKIP_ERRORS_KEY, WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
210
211    Set<ServerName> serverNames = new HashSet<>();
212    Path logsDirPath = new Path(this.rootDir, HConstants.HREGION_LOGDIR_NAME);
213
214    do {
215      if (services.isStopped()) {
216        LOG.warn("Master stopped while trying to get failed servers.");
217        break;
218      }
219      try {
220        if (!this.fs.exists(logsDirPath)) return serverNames;
221        FileStatus[] logFolders = CommonFSUtils.listStatus(this.fs, logsDirPath, null);
222        // Get online servers after getting log folders to avoid log folder deletion of newly
223        // checked in region servers . see HBASE-5916
224        Set<ServerName> onlineServers = services.getServerManager().getOnlineServers().keySet();
225
226        if (logFolders == null || logFolders.length == 0) {
227          LOG.debug("No log files to split, proceeding...");
228          return serverNames;
229        }
230        for (FileStatus status : logFolders) {
231          FileStatus[] curLogFiles = CommonFSUtils.listStatus(this.fs, status.getPath(), null);
232          if (curLogFiles == null || curLogFiles.length == 0) {
233            // Empty log folder. No recovery needed
234            continue;
235          }
236          final ServerName serverName =
237            AbstractFSWALProvider.getServerNameFromWALDirectoryName(status.getPath());
238          if (null == serverName) {
239            LOG.warn("Log folder " + status.getPath() + " doesn't look like its name includes a "
240              + "region server name; leaving in place. If you see later errors about missing "
241              + "write ahead logs they may be saved in this location.");
242          } else if (!onlineServers.contains(serverName)) {
243            LOG.info("Log folder " + status.getPath() + " doesn't belong "
244              + "to a known region server, splitting");
245            serverNames.add(serverName);
246          } else {
247            LOG.info("Log folder " + status.getPath() + " belongs to an existing region server");
248          }
249        }
250        retrySplitting = false;
251      } catch (IOException ioe) {
252        LOG.warn("Failed getting failed servers to be recovered.", ioe);
253        if (!checkFileSystem()) {
254          LOG.warn("Bad Filesystem, exiting");
255          Runtime.getRuntime().halt(1);
256        }
257        try {
258          if (retrySplitting) {
259            Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000));
260          }
261        } catch (InterruptedException e) {
262          LOG.warn("Interrupted, aborting since cannot return w/o splitting");
263          Thread.currentThread().interrupt();
264          retrySplitting = false;
265          Runtime.getRuntime().halt(1);
266        }
267      }
268    } while (retrySplitting);
269
270    return serverNames;
271  }
272
273  public void splitLog(final ServerName serverName) throws IOException {
274    splitLog(Collections.<ServerName> singleton(serverName));
275  }
276
277  /**
278   * Specialized method to handle the splitting for meta WAL
279   * @param serverName logs belonging to this server will be split
280   */
281  public void splitMetaLog(final ServerName serverName) throws IOException {
282    splitMetaLog(Collections.<ServerName> singleton(serverName));
283  }
284
285  /**
286   * Specialized method to handle the splitting for meta WAL
287   * @param serverNames logs belonging to these servers will be split
288   */
289  public void splitMetaLog(final Set<ServerName> serverNames) throws IOException {
290    splitLog(serverNames, META_FILTER);
291  }
292
293  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UL_UNRELEASED_LOCK",
294      justification = "We only release this lock when we set it. Updates to code "
295        + "that uses it should verify use of the guard boolean.")
296  List<Path> getLogDirs(final Set<ServerName> serverNames) throws IOException {
297    List<Path> logDirs = new ArrayList<>();
298    boolean needReleaseLock = false;
299    if (!this.services.isInitialized()) {
300      // during master initialization, we could have multiple places splitting a same wal
301      // XXX: Does this still exist after we move to proc-v2?
302      this.splitLogLock.lock();
303      needReleaseLock = true;
304    }
305    try {
306      for (ServerName serverName : serverNames) {
307        Path logDir =
308          new Path(this.rootDir, AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
309        Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
310        // Rename the directory so a rogue RS doesn't create more WALs
311        if (fs.exists(logDir)) {
312          if (!this.fs.rename(logDir, splitDir)) {
313            throw new IOException("Failed fs.rename for log split: " + logDir);
314          }
315          logDir = splitDir;
316          LOG.debug("Renamed region directory: " + splitDir);
317        } else if (!fs.exists(splitDir)) {
318          LOG.info("Log dir for server " + serverName + " does not exist");
319          continue;
320        }
321        logDirs.add(splitDir);
322      }
323    } catch (IOException ioe) {
324      if (!checkFileSystem()) {
325        this.services.abort("Aborting due to filesystem unavailable", ioe);
326        throw ioe;
327      }
328    } finally {
329      if (needReleaseLock) {
330        this.splitLogLock.unlock();
331      }
332    }
333    return logDirs;
334  }
335
336  public void splitLog(final Set<ServerName> serverNames) throws IOException {
337    splitLog(serverNames, NON_META_FILTER);
338  }
339
340  /**
341   * This method is the base split method that splits WAL files matching a filter. Callers should
342   * pass the appropriate filter for meta and non-meta WALs.
343   * @param serverNames logs belonging to these servers will be split; this will rename the log
344   *                    directory out from under a soft-failed server
345   */
346  public void splitLog(final Set<ServerName> serverNames, PathFilter filter) throws IOException {
347    long splitTime = 0, splitLogSize = 0;
348    List<Path> logDirs = getLogDirs(serverNames);
349
350    splitLogManager.handleDeadWorkers(serverNames);
351    splitTime = EnvironmentEdgeManager.currentTime();
352    splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter);
353    splitTime = EnvironmentEdgeManager.currentTime() - splitTime;
354
355    if (this.metricsMasterFilesystem != null) {
356      if (filter == META_FILTER) {
357        this.metricsMasterFilesystem.addMetaWALSplit(splitTime, splitLogSize);
358      } else {
359        this.metricsMasterFilesystem.addSplit(splitTime, splitLogSize);
360      }
361    }
362  }
363
364  /**
365   * The hbase:meta region may OPEN and CLOSE without issue on a server and then move elsewhere. On
366   * CLOSE, the WAL for the hbase:meta table may not be archived yet (The WAL is only needed if
367   * hbase:meta did not close cleanaly). Since meta region is no long on this server, the
368   * ServerCrashProcedure won't split these leftover hbase:meta WALs, just leaving them in the WAL
369   * splitting dir. If we try to delete the WAL splitting for the server, it fail since the dir is
370   * not totally empty. We can safely archive these hbase:meta log; then the WAL dir can be deleted.
371   * @param serverName the server to archive meta log
372   */
373  public void archiveMetaLog(final ServerName serverName) {
374    try {
375      Path logDir =
376        new Path(this.rootDir, AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
377      Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
378      if (fs.exists(splitDir)) {
379        FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
380        if (logfiles != null) {
381          for (FileStatus status : logfiles) {
382            if (!status.isDir()) {
383              Path newPath = AbstractFSWAL.getWALArchivePath(this.oldLogDir, status.getPath());
384              if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
385                LOG.warn("Unable to move  " + status.getPath() + " to " + newPath);
386              } else {
387                LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);
388              }
389            }
390          }
391        }
392        if (!fs.delete(splitDir, false)) {
393          LOG.warn("Unable to delete log dir. Ignoring. " + splitDir);
394        }
395      }
396    } catch (IOException ie) {
397      LOG.warn("Failed archiving meta log for server " + serverName, ie);
398    }
399  }
400}