Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-4026][Streaming] Write ahead log management #2882

Closed
wants to merge 20 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
172358d
Pulled WriteAheadLog-related stuff from tdas/spark/tree/driver-ha-wor…
tdas Oct 21, 2014
5182ffb
Added documentation
harishreedharan Oct 21, 2014
b06be2b
Adding missing license.
tdas Oct 21, 2014
4ab602a
Refactored write ahead stuff from streaming.storage to streaming.util
tdas Oct 21, 2014
5c70d1f
Remove underlying stream from the WALWriter.
harishreedharan Oct 22, 2014
edcbee1
Tests reading and writing data using writers now use Minicluster.
harishreedharan Oct 9, 2014
b4be0c1
Remove unused method
harishreedharan Oct 22, 2014
587b876
Fix broken test. Call getFileSystem only from synchronized method.
harishreedharan Oct 22, 2014
7e40e56
Restore old build directory after tests
harishreedharan Oct 22, 2014
ef8db09
Merge pull request #17 from harishreedharan/driver-ha-wal
tdas Oct 22, 2014
5ff90ee
Fix tests to not ignore ordering and also assert all data is present
harishreedharan Oct 22, 2014
82ce56e
Fix file ordering issue in WALManager tests
harishreedharan Oct 22, 2014
eb356ca
Merge pull request #18 from harishreedharan/driver-ha-wal
tdas Oct 23, 2014
4705fff
Sort listed files by name. Use local files for WAL tests.
harishreedharan Oct 23, 2014
3881706
Merge pull request #19 from harishreedharan/driver-ha-wal
tdas Oct 23, 2014
9514dc8
Added unit tests to test reading of corrupted data and other minor edits
tdas Oct 23, 2014
a317a4d
Directory deletion should not fail tests
harishreedharan Oct 23, 2014
d29fddd
Merge pull request #20 from harishreedharan/driver-ha-wal
tdas Oct 23, 2014
55514e2
Minor changes based on PR comments.
tdas Oct 24, 2014
e4bee20
Removed synchronized, Path.getFileSystem is threadsafe
tdas Oct 24, 2014
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.streaming.util

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._

private[streaming] object HdfsUtils {

def getOutputStream(path: String, conf: Configuration): FSDataOutputStream = {
val dfsPath = new Path(path)
val dfs = getFileSystemForPath(dfsPath, conf)
// If the file exists and we have append support, append instead of creating a new file
val stream: FSDataOutputStream = {
if (dfs.isFile(dfsPath)) {
if (conf.getBoolean("hdfs.append.support", false)) {
dfs.append(dfsPath)
} else {
throw new IllegalStateException("File exists and there is no append support!")
}
} else {
dfs.create(dfsPath)
}
}
stream
}

def getInputStream(path: String, conf: Configuration): FSDataInputStream = {
val dfsPath = new Path(path)
val dfs = getFileSystemForPath(dfsPath, conf)
val instream = dfs.open(dfsPath)
instream
}

def checkState(state: Boolean, errorMsg: => String) {
if (!state) {
throw new IllegalStateException(errorMsg)
}
}

def getBlockLocations(path: String, conf: Configuration): Option[Array[String]] = {
val dfsPath = new Path(path)
val dfs = getFileSystemForPath(dfsPath, conf)
val fileStatus = dfs.getFileStatus(dfsPath)
val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen))
blockLocs.map(_.flatMap(_.getHosts))
}

def getFileSystemForPath(path: Path, conf: Configuration): FileSystem = {
// For local file systems, return the raw loca file system, such calls to flush()
// actually flushes the stream.
val fs = path.getFileSystem(conf)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on the old comment, does this need synchronization?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@harishreedharan Can you elaborate on why getFileSystem is not thread-safe? References?
And if it is indeed not thread-safe, then doing synchronization here does not solve the problem because other threads in spark could be access getFileSystem at the same time.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@aarondav, is this that same file system issue that you mentioned to me?

fs match {
case localFs: LocalFileSystem => localFs.getRawFileSystem
case _ => fs
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.streaming.util

/** Class for representing a segment of data in a write ahead log file */
private[streaming] case class WriteAheadLogFileSegment (path: String, offset: Long, length: Int)
Original file line number Diff line number Diff line change
@@ -0,0 +1,224 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.streaming.util

import java.nio.ByteBuffer

import scala.collection.mutable.ArrayBuffer
import scala.concurrent.{ExecutionContext, Future}

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.permission.FsPermission
import org.apache.spark.Logging
import org.apache.spark.util.Utils
import WriteAheadLogManager._

/**
* This class manages write ahead log files.
* - Writes records (bytebuffers) to periodically rotating log files.
* - Recovers the log files and the reads the recovered records upon failures.
* - Cleans up old log files.
*
* Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write
* and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read.
*
* @param logDirectory Directory when rotating log files will be created.
* @param hadoopConf Hadoop configuration for reading/writing log files.
* @param rollingIntervalSecs The interval in seconds with which logs will be rolled over.
* Default is one minute.
* @param maxFailures Max number of failures that is tolerated for every attempt to write to log.
* Default is three.
* @param callerName Optional name of the class who is using this manager.
* @param clock Optional clock that is used to check for rotation interval.
*/
private[streaming] class WriteAheadLogManager(
logDirectory: String,
hadoopConf: Configuration,
rollingIntervalSecs: Int = 60,
maxFailures: Int = 3,
callerName: String = "",
clock: Clock = new SystemClock
) extends Logging {

private val pastLogs = new ArrayBuffer[LogInfo]
private val callerNameTag =
if (callerName.nonEmpty) s" for $callerName" else ""
private val threadpoolName = s"WriteAheadLogManager $callerNameTag"
implicit private val executionContext = ExecutionContext.fromExecutorService(
Utils.newDaemonFixedThreadPool(1, threadpoolName))
override protected val logName = s"WriteAheadLogManager $callerNameTag"

private var currentLogPath: Option[String] = None
private var currentLogWriter: WriteAheadLogWriter = null
private var currentLogWriterStartTime: Long = -1L
private var currentLogWriterStopTime: Long = -1L

initializeOrRecover()

/**
* Write a byte buffer to the log file. This method synchronously writes the data in the
* ByteBuffer to HDFS. When this method returns, the data is guaranteed to have been flushed
* to HDFS, and will be available for readers to read.
*/
def writeToLog(byteBuffer: ByteBuffer): WriteAheadLogFileSegment = synchronized {
var fileSegment: WriteAheadLogFileSegment = null
var failures = 0
var lastException: Exception = null
var succeeded = false
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that succeeded implies fileSegment != null and vice-versa, so you probably don't need this variable.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had thought so, and I usually do it. But for reading I felt its easier to understand if (!succeeded ... . Am happy to change it.

while (!succeeded && failures < maxFailures) {
try {
fileSegment = getLogWriter(clock.currentTime).write(byteBuffer)
succeeded = true
} catch {
case ex: Exception =>
lastException = ex
logWarning("Failed to write to write ahead log")
resetWriter()
failures += 1
}
}
if (fileSegment == null) {
logError(s"Failed to write to write ahead log after $failures failures")
throw lastException
}
fileSegment
}

/**
* Read all the existing logs from the log directory.
*
* Note that this is typically called when the caller is initializing and wants
* to recover past state from the write ahead logs (that is, before making any writes).
* If this is called after writes have been made using this manager, then it may not return
* the latest the records. This does not deal with currently active log files, and
* hence the implementation is kept simple.
*/
def readFromLog(): Iterator[ByteBuffer] = synchronized {
val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath
logInfo("Reading from the logs: " + logFilesToRead.mkString("\n"))
logFilesToRead.iterator.map { file =>
logDebug(s"Creating log reader with $file")
new WriteAheadLogReader(file, hadoopConf)
} flatMap { x => x }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you can just write .flatten instead of flatMapping over identity.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Tried, for some reason cant do it. Iterator does not have flatten.
http://www.scala-lang.org/api/current/index.html#scala.collection.Iterator

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, that makes sense, actually: flatMap(identity) on an iterator here is the equivalent of Python's itertools.chain, whereas flatten might imply materializing the flattened collection, which doesn't make sense for an iterator.

}

/**
* Delete the log files that are older than the threshold time.
*
* Its important to note that the threshold time is based on the time stamps used in the log
* files, which is usually based on the local system time. So if there is coordination necessary
* between the node calculating the threshTime (say, driver node), and the local system time
* (say, worker node), the caller has to take account of possible time skew.
*/
def cleanupOldLogs(threshTime: Long): Unit = {
val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } }
logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " +
s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}")

def deleteFiles() {
oldLogFiles.foreach { logInfo =>
try {
val path = new Path(logInfo.path)
val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf)
fs.delete(path, true)
synchronized { pastLogs -= logInfo }
logDebug(s"Cleared log file $logInfo")
} catch {
case ex: Exception =>
logWarning(s"Error clearing write ahead log file $logInfo", ex)
}
}
logInfo(s"Cleared log files in $logDirectory older than $threshTime")
}
if (!executionContext.isShutdown) {
Future { deleteFiles() }
}
}

/** Stop the manager, close any open log writer */
def stop(): Unit = synchronized {
if (currentLogWriter != null) {
currentLogWriter.close()
}
executionContext.shutdown()
logInfo("Stopped write ahead log manager")
}

/** Get the current log writer while taking care of rotation */
private def getLogWriter(currentTime: Long): WriteAheadLogWriter = synchronized {
if (currentLogWriter == null || currentTime > currentLogWriterStopTime) {
resetWriter()
currentLogPath.foreach {
pastLogs += LogInfo(currentLogWriterStartTime, currentLogWriterStopTime, _)
}
currentLogWriterStartTime = currentTime
currentLogWriterStopTime = currentTime + (rollingIntervalSecs * 1000)
val newLogPath = new Path(logDirectory,
timeToLogFile(currentLogWriterStartTime, currentLogWriterStopTime))
currentLogPath = Some(newLogPath.toString)
currentLogWriter = new WriteAheadLogWriter(currentLogPath.get, hadoopConf)
}
currentLogWriter
}

/** Initialize the log directory or recover existing logs inside the directory */
private def initializeOrRecover(): Unit = synchronized {
val logDirectoryPath = new Path(logDirectory)
val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)

if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) {
val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath })
pastLogs.clear()
pastLogs ++= logFileInfo
logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory")
logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}")
}
}

private def resetWriter(): Unit = synchronized {
if (currentLogWriter != null) {
currentLogWriter.close()
currentLogWriter = null
}
}
}

private[util] object WriteAheadLogManager {

case class LogInfo(startTime: Long, endTime: Long, path: String)

val logFileRegex = """log-(\d+)-(\d+)""".r

def timeToLogFile(startTime: Long, stopTime: Long): String = {
s"log-$startTime-$stopTime"
}

/** Convert a sequence of files to a sequence of sorted LogInfo objects */
def logFilesTologInfo(files: Seq[Path]): Seq[LogInfo] = {
files.flatMap { file =>
logFileRegex.findFirstIn(file.getName()) match {
case Some(logFileRegex(startTimeStr, stopTimeStr)) =>
val startTime = startTimeStr.toLong
val stopTime = stopTimeStr.toLong
Some(LogInfo(startTime, stopTime, file.toString))
case None =>
None
}
}.sortBy { _.startTime }
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.streaming.util

import java.io.Closeable
import java.nio.ByteBuffer

import org.apache.hadoop.conf.Configuration

/**
* A random access reader for reading write ahead log files written using
* [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. Given the file segment info,
* this reads the record (bytebuffer) from the log file.
*/
private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configuration)
extends Closeable {

private val instream = HdfsUtils.getInputStream(path, conf)
private var closed = false

def read(segment: WriteAheadLogFileSegment): ByteBuffer = synchronized {
assertOpen()
instream.seek(segment.offset)
val nextLength = instream.readInt()
HdfsUtils.checkState(nextLength == segment.length,
s"Expected message length to be ${segment.length}, but was $nextLength")
val buffer = new Array[Byte](nextLength)
instream.readFully(buffer)
ByteBuffer.wrap(buffer)
}

override def close(): Unit = synchronized {
closed = true
instream.close()
}

private def assertOpen() {
HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the file.")
}
}

Loading