|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.spark.streaming.util |
| 18 | + |
| 19 | +import java.nio.ByteBuffer |
| 20 | + |
| 21 | +import scala.collection.mutable.ArrayBuffer |
| 22 | +import scala.concurrent.{ExecutionContext, Future} |
| 23 | + |
| 24 | +import org.apache.hadoop.conf.Configuration |
| 25 | +import org.apache.hadoop.fs.Path |
| 26 | +import org.apache.hadoop.fs.permission.FsPermission |
| 27 | +import org.apache.spark.Logging |
| 28 | +import org.apache.spark.util.Utils |
| 29 | +import WriteAheadLogManager._ |
| 30 | + |
| 31 | +/** |
| 32 | + * This class manages write ahead log files. |
| 33 | + * - Writes records (bytebuffers) to periodically rotating log files. |
| 34 | + * - Recovers the log files and the reads the recovered records upon failures. |
| 35 | + * - Cleans up old log files. |
| 36 | + * |
| 37 | + * Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write |
| 38 | + * and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read. |
| 39 | + * |
| 40 | + * @param logDirectory Directory when rotating log files will be created. |
| 41 | + * @param hadoopConf Hadoop configuration for reading/writing log files. |
| 42 | + * @param rollingIntervalSecs The interval in seconds with which logs will be rolled over. |
| 43 | + * Default is one minute. |
| 44 | + * @param maxFailures Max number of failures that is tolerated for every attempt to write to log. |
| 45 | + * Default is three. |
| 46 | + * @param callerName Optional name of the class who is using this manager. |
| 47 | + * @param clock Optional clock that is used to check for rotation interval. |
| 48 | + */ |
| 49 | +private[streaming] class WriteAheadLogManager( |
| 50 | + logDirectory: String, |
| 51 | + hadoopConf: Configuration, |
| 52 | + rollingIntervalSecs: Int = 60, |
| 53 | + maxFailures: Int = 3, |
| 54 | + callerName: String = "", |
| 55 | + clock: Clock = new SystemClock |
| 56 | + ) extends Logging { |
| 57 | + |
| 58 | + private val pastLogs = new ArrayBuffer[LogInfo] |
| 59 | + private val callerNameTag = |
| 60 | + if (callerName.nonEmpty) s" for $callerName" else "" |
| 61 | + private val threadpoolName = s"WriteAheadLogManager $callerNameTag" |
| 62 | + implicit private val executionContext = ExecutionContext.fromExecutorService( |
| 63 | + Utils.newDaemonFixedThreadPool(1, threadpoolName)) |
| 64 | + override protected val logName = s"WriteAheadLogManager $callerNameTag" |
| 65 | + |
| 66 | + private var currentLogPath: Option[String] = None |
| 67 | + private var currentLogWriter: WriteAheadLogWriter = null |
| 68 | + private var currentLogWriterStartTime: Long = -1L |
| 69 | + private var currentLogWriterStopTime: Long = -1L |
| 70 | + |
| 71 | + initializeOrRecover() |
| 72 | + |
| 73 | + /** |
| 74 | + * Write a byte buffer to the log file. This method synchronously writes the data in the |
| 75 | + * ByteBuffer to HDFS. When this method returns, the data is guaranteed to have been flushed |
| 76 | + * to HDFS, and will be available for readers to read. |
| 77 | + */ |
| 78 | + def writeToLog(byteBuffer: ByteBuffer): WriteAheadLogFileSegment = synchronized { |
| 79 | + var fileSegment: WriteAheadLogFileSegment = null |
| 80 | + var failures = 0 |
| 81 | + var lastException: Exception = null |
| 82 | + var succeeded = false |
| 83 | + while (!succeeded && failures < maxFailures) { |
| 84 | + try { |
| 85 | + fileSegment = getLogWriter(clock.currentTime).write(byteBuffer) |
| 86 | + succeeded = true |
| 87 | + } catch { |
| 88 | + case ex: Exception => |
| 89 | + lastException = ex |
| 90 | + logWarning("Failed to write to write ahead log") |
| 91 | + resetWriter() |
| 92 | + failures += 1 |
| 93 | + } |
| 94 | + } |
| 95 | + if (fileSegment == null) { |
| 96 | + logError(s"Failed to write to write ahead log after $failures failures") |
| 97 | + throw lastException |
| 98 | + } |
| 99 | + fileSegment |
| 100 | + } |
| 101 | + |
| 102 | + /** |
| 103 | + * Read all the existing logs from the log directory. |
| 104 | + * |
| 105 | + * Note that this is typically called when the caller is initializing and wants |
| 106 | + * to recover past state from the write ahead logs (that is, before making any writes). |
| 107 | + * If this is called after writes have been made using this manager, then it may not return |
| 108 | + * the latest the records. This does not deal with currently active log files, and |
| 109 | + * hence the implementation is kept simple. |
| 110 | + */ |
| 111 | + def readFromLog(): Iterator[ByteBuffer] = synchronized { |
| 112 | + val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath |
| 113 | + logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) |
| 114 | + logFilesToRead.iterator.map { file => |
| 115 | + logDebug(s"Creating log reader with $file") |
| 116 | + new WriteAheadLogReader(file, hadoopConf) |
| 117 | + } flatMap { x => x } |
| 118 | + } |
| 119 | + |
| 120 | + /** |
| 121 | + * Delete the log files that are older than the threshold time. |
| 122 | + * |
| 123 | + * Its important to note that the threshold time is based on the time stamps used in the log |
| 124 | + * files, which is usually based on the local system time. So if there is coordination necessary |
| 125 | + * between the node calculating the threshTime (say, driver node), and the local system time |
| 126 | + * (say, worker node), the caller has to take account of possible time skew. |
| 127 | + */ |
| 128 | + def cleanupOldLogs(threshTime: Long): Unit = { |
| 129 | + val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } |
| 130 | + logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + |
| 131 | + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") |
| 132 | + |
| 133 | + def deleteFiles() { |
| 134 | + oldLogFiles.foreach { logInfo => |
| 135 | + try { |
| 136 | + val path = new Path(logInfo.path) |
| 137 | + val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf) |
| 138 | + fs.delete(path, true) |
| 139 | + synchronized { pastLogs -= logInfo } |
| 140 | + logDebug(s"Cleared log file $logInfo") |
| 141 | + } catch { |
| 142 | + case ex: Exception => |
| 143 | + logWarning(s"Error clearing write ahead log file $logInfo", ex) |
| 144 | + } |
| 145 | + } |
| 146 | + logInfo(s"Cleared log files in $logDirectory older than $threshTime") |
| 147 | + } |
| 148 | + if (!executionContext.isShutdown) { |
| 149 | + Future { deleteFiles() } |
| 150 | + } |
| 151 | + } |
| 152 | + |
| 153 | + /** Stop the manager, close any open log writer */ |
| 154 | + def stop(): Unit = synchronized { |
| 155 | + if (currentLogWriter != null) { |
| 156 | + currentLogWriter.close() |
| 157 | + } |
| 158 | + executionContext.shutdown() |
| 159 | + logInfo("Stopped write ahead log manager") |
| 160 | + } |
| 161 | + |
| 162 | + /** Get the current log writer while taking care of rotation */ |
| 163 | + private def getLogWriter(currentTime: Long): WriteAheadLogWriter = synchronized { |
| 164 | + if (currentLogWriter == null || currentTime > currentLogWriterStopTime) { |
| 165 | + resetWriter() |
| 166 | + currentLogPath.foreach { |
| 167 | + pastLogs += LogInfo(currentLogWriterStartTime, currentLogWriterStopTime, _) |
| 168 | + } |
| 169 | + currentLogWriterStartTime = currentTime |
| 170 | + currentLogWriterStopTime = currentTime + (rollingIntervalSecs * 1000) |
| 171 | + val newLogPath = new Path(logDirectory, |
| 172 | + timeToLogFile(currentLogWriterStartTime, currentLogWriterStopTime)) |
| 173 | + currentLogPath = Some(newLogPath.toString) |
| 174 | + currentLogWriter = new WriteAheadLogWriter(currentLogPath.get, hadoopConf) |
| 175 | + } |
| 176 | + currentLogWriter |
| 177 | + } |
| 178 | + |
| 179 | + /** Initialize the log directory or recover existing logs inside the directory */ |
| 180 | + private def initializeOrRecover(): Unit = synchronized { |
| 181 | + val logDirectoryPath = new Path(logDirectory) |
| 182 | + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) |
| 183 | + |
| 184 | + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { |
| 185 | + val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath }) |
| 186 | + pastLogs.clear() |
| 187 | + pastLogs ++= logFileInfo |
| 188 | + logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory") |
| 189 | + logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") |
| 190 | + } |
| 191 | + } |
| 192 | + |
| 193 | + private def resetWriter(): Unit = synchronized { |
| 194 | + if (currentLogWriter != null) { |
| 195 | + currentLogWriter.close() |
| 196 | + currentLogWriter = null |
| 197 | + } |
| 198 | + } |
| 199 | +} |
| 200 | + |
| 201 | +private[util] object WriteAheadLogManager { |
| 202 | + |
| 203 | + case class LogInfo(startTime: Long, endTime: Long, path: String) |
| 204 | + |
| 205 | + val logFileRegex = """log-(\d+)-(\d+)""".r |
| 206 | + |
| 207 | + def timeToLogFile(startTime: Long, stopTime: Long): String = { |
| 208 | + s"log-$startTime-$stopTime" |
| 209 | + } |
| 210 | + |
| 211 | + /** Convert a sequence of files to a sequence of sorted LogInfo objects */ |
| 212 | + def logFilesTologInfo(files: Seq[Path]): Seq[LogInfo] = { |
| 213 | + files.flatMap { file => |
| 214 | + logFileRegex.findFirstIn(file.getName()) match { |
| 215 | + case Some(logFileRegex(startTimeStr, stopTimeStr)) => |
| 216 | + val startTime = startTimeStr.toLong |
| 217 | + val stopTime = stopTimeStr.toLong |
| 218 | + Some(LogInfo(startTime, stopTime, file.toString)) |
| 219 | + case None => |
| 220 | + None |
| 221 | + } |
| 222 | + }.sortBy { _.startTime } |
| 223 | + } |
| 224 | +} |
0 commit comments