@@ -13,6 +13,24 @@ import org.apache.spark.streaming.storage.WriteAheadLogManager._
13
13
import org .apache .spark .streaming .util .{Clock , SystemClock }
14
14
import org .apache .spark .util .Utils
15
15
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
+ */
16
34
private [streaming] class WriteAheadLogManager (
17
35
logDirectory : String ,
18
36
hadoopConf : Configuration ,
@@ -37,6 +55,7 @@ private[streaming] class WriteAheadLogManager(
37
55
38
56
initializeOrRecover()
39
57
58
+ /** Write a byte buffer to the log file */
40
59
def writeToLog (byteBuffer : ByteBuffer ): FileSegment = synchronized {
41
60
var fileSegment : FileSegment = null
42
61
var failures = 0
@@ -49,17 +68,27 @@ private[streaming] class WriteAheadLogManager(
49
68
} catch {
50
69
case ex : Exception =>
51
70
lastException = ex
52
- logWarning(" Failed to ... " )
71
+ logWarning(" Failed to write to write ahead log " )
53
72
resetWriter()
54
73
failures += 1
55
74
}
56
75
}
57
76
if (fileSegment == null ) {
77
+ logError(s " Failed to write to write ahead log after $failures failures " )
58
78
throw lastException
59
79
}
60
80
fileSegment
61
81
}
62
82
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
+ */
63
92
def readFromLog (): Iterator [ByteBuffer ] = synchronized {
64
93
val logFilesToRead = pastLogs.map{ _.path} ++ Option (currentLogPath)
65
94
logInfo(" Reading from the logs: " + logFilesToRead.mkString(" \n " ))
@@ -73,7 +102,7 @@ private[streaming] class WriteAheadLogManager(
73
102
* Delete the log files that are older than the threshold time.
74
103
*
75
104
* 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
77
106
* between the node calculating the threshTime (say, driver node), and the local system time
78
107
* (say, worker node), the caller has to take account of possible time skew.
79
108
*/
@@ -92,7 +121,7 @@ private[streaming] class WriteAheadLogManager(
92
121
logDebug(s " Cleared log file $logInfo" )
93
122
} catch {
94
123
case ex : Exception =>
95
- logWarning(s " Error clearing log file $logInfo" , ex)
124
+ logWarning(s " Error clearing write ahead log file $logInfo" , ex)
96
125
}
97
126
}
98
127
logInfo(s " Cleared log files in $logDirectory older than $threshTime" )
@@ -102,14 +131,16 @@ private[streaming] class WriteAheadLogManager(
102
131
}
103
132
}
104
133
134
+ /** Stop the manager, close any open log writer */
105
135
def stop (): Unit = synchronized {
106
136
if (currentLogWriter != null ) {
107
137
currentLogWriter.close()
108
138
}
109
139
executionContext.shutdown()
110
- logInfo(" Stopped log manager" )
140
+ logInfo(" Stopped write ahead log manager" )
111
141
}
112
142
143
+ /** Get the current log writer while taking care of rotation */
113
144
private def getLogWriter (currentTime : Long ): WriteAheadLogWriter = synchronized {
114
145
if (currentLogWriter == null || currentTime > currentLogWriterStopTime) {
115
146
resetWriter()
@@ -126,6 +157,7 @@ private[streaming] class WriteAheadLogManager(
126
157
currentLogWriter
127
158
}
128
159
160
+ /** Initialize the log directory or recover existing logs inside the directory */
129
161
private def initializeOrRecover (): Unit = synchronized {
130
162
val logDirectoryPath = new Path (logDirectory)
131
163
val fileSystem = logDirectoryPath.getFileSystem(hadoopConf)
@@ -134,12 +166,12 @@ private[streaming] class WriteAheadLogManager(
134
166
val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath })
135
167
pastLogs.clear()
136
168
pastLogs ++= logFileInfo
137
- logInfo(s " Recovered ${logFileInfo.size} log files from $logDirectory" )
169
+ logInfo(s " Recovered ${logFileInfo.size} write ahead log files from $logDirectory" )
138
170
logDebug(s " Recovered files are: \n ${logFileInfo.map(_.path).mkString(" \n " )}" )
139
171
} else {
140
172
fileSystem.mkdirs(logDirectoryPath,
141
173
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 " )
143
175
}
144
176
}
145
177
0 commit comments