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}