@@ -13,6 +13,24 @@ import org.apache.spark.streaming.storage.WriteAheadLogManager._
1313import org .apache .spark .streaming .util .{Clock , SystemClock }
1414import org .apache .spark .util .Utils
1515
16+ /**
17+ * This class manages write ahead log files.
18+ * - Writes records (bytebuffers) to periodically rotating log files.
19+ * - Recovers the log files and the reads the recovered records upon failures.
20+ * - Cleans up old log files.
21+ *
22+ * Uses [[org.apache.spark.streaming.storage.WriteAheadLogWriter ]] to write
23+ * and [[org.apache.spark.streaming.storage.WriteAheadLogReader ]] to read.
24+ *
25+ *@param logDirectory Directory when rotating log files will be created.
26+ * @param hadoopConf Hadoop configuration for reading/writing log files.
27+ * @param rollingIntervalSecs The interval in seconds with which logs will be rolled over.
28+ * Default is one minute.
29+ * @param maxFailures Max number of failures that is tolerated for every attempt to write to log.
30+ * Default is three.
31+ * @param callerName Optional name of the class who is using this manager.
32+ * @param clock Optional clock that is used to check for rotation interval.
33+ */
1634private [streaming] class WriteAheadLogManager (
1735 logDirectory : String ,
1836 hadoopConf : Configuration ,
@@ -37,6 +55,7 @@ private[streaming] class WriteAheadLogManager(
3755
3856 initializeOrRecover()
3957
58+ /** Write a byte buffer to the log file */
4059 def writeToLog (byteBuffer : ByteBuffer ): FileSegment = synchronized {
4160 var fileSegment : FileSegment = null
4261 var failures = 0
@@ -49,17 +68,27 @@ private[streaming] class WriteAheadLogManager(
4968 } catch {
5069 case ex : Exception =>
5170 lastException = ex
52- logWarning(" Failed to ... " )
71+ logWarning(" Failed to write to write ahead log " )
5372 resetWriter()
5473 failures += 1
5574 }
5675 }
5776 if (fileSegment == null ) {
77+ logError(s " Failed to write to write ahead log after $failures failures " )
5878 throw lastException
5979 }
6080 fileSegment
6181 }
6282
83+ /**
84+ * Read all the existing logs from the log directory.
85+ *
86+ * Note that this is typically called when the caller is initializing and wants
87+ * to recover past state from the write ahead logs (that is, before making any writes).
88+ * If this is called after writes have been made using this manager, then it may not return
89+ * the latest the records. This does not deal with currently active log files, and
90+ * hence the implementation is kept simple.
91+ */
6392 def readFromLog (): Iterator [ByteBuffer ] = synchronized {
6493 val logFilesToRead = pastLogs.map{ _.path} ++ Option (currentLogPath)
6594 logInfo(" Reading from the logs: " + logFilesToRead.mkString(" \n " ))
@@ -73,7 +102,7 @@ private[streaming] class WriteAheadLogManager(
73102 * Delete the log files that are older than the threshold time.
74103 *
75104 * Its important to note that the threshold time is based on the time stamps used in the log
76- * files, and is therefore based on the local system time. So if there is coordination necessary
105+ * files, which is usually based on the local system time. So if there is coordination necessary
77106 * between the node calculating the threshTime (say, driver node), and the local system time
78107 * (say, worker node), the caller has to take account of possible time skew.
79108 */
@@ -92,7 +121,7 @@ private[streaming] class WriteAheadLogManager(
92121 logDebug(s " Cleared log file $logInfo" )
93122 } catch {
94123 case ex : Exception =>
95- logWarning(s " Error clearing log file $logInfo" , ex)
124+ logWarning(s " Error clearing write ahead log file $logInfo" , ex)
96125 }
97126 }
98127 logInfo(s " Cleared log files in $logDirectory older than $threshTime" )
@@ -102,14 +131,16 @@ private[streaming] class WriteAheadLogManager(
102131 }
103132 }
104133
134+ /** Stop the manager, close any open log writer */
105135 def stop (): Unit = synchronized {
106136 if (currentLogWriter != null ) {
107137 currentLogWriter.close()
108138 }
109139 executionContext.shutdown()
110- logInfo(" Stopped log manager" )
140+ logInfo(" Stopped write ahead log manager" )
111141 }
112142
143+ /** Get the current log writer while taking care of rotation */
113144 private def getLogWriter (currentTime : Long ): WriteAheadLogWriter = synchronized {
114145 if (currentLogWriter == null || currentTime > currentLogWriterStopTime) {
115146 resetWriter()
@@ -126,6 +157,7 @@ private[streaming] class WriteAheadLogManager(
126157 currentLogWriter
127158 }
128159
160+ /** Initialize the log directory or recover existing logs inside the directory */
129161 private def initializeOrRecover (): Unit = synchronized {
130162 val logDirectoryPath = new Path (logDirectory)
131163 val fileSystem = logDirectoryPath.getFileSystem(hadoopConf)
@@ -134,12 +166,12 @@ private[streaming] class WriteAheadLogManager(
134166 val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath })
135167 pastLogs.clear()
136168 pastLogs ++= logFileInfo
137- logInfo(s " Recovered ${logFileInfo.size} log files from $logDirectory" )
169+ logInfo(s " Recovered ${logFileInfo.size} write ahead log files from $logDirectory" )
138170 logDebug(s " Recovered files are: \n ${logFileInfo.map(_.path).mkString(" \n " )}" )
139171 } else {
140172 fileSystem.mkdirs(logDirectoryPath,
141173 FsPermission .createImmutable(Integer .parseInt(" 770" , 8 ).toShort))
142- logInfo(s " Created ${logDirectory} for log files " )
174+ logInfo(s " Created ${logDirectory} for write ahead log files " )
143175 }
144176 }
145177
0 commit comments