-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-7056][Streaming] Make the Write Ahead Log pluggable #5645
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
Changes from 11 commits
7dd2d4b
09bc6fe
754fbf8
837c4f5
bce5e75
84ce469
86abcb1
9310cbf
e0d19fb
1a32a4b
d7cd15b
b65e155
bde26b1
569a416
c2bc738
2c431fd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,59 @@ | ||
| /* | ||
| * 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 java.util.Iterator; | ||
|
|
||
| /** | ||
| * Interface representing a write ahead log (aka journal) that is used by Spark Streaming to | ||
| * save the received data (by receivers) and associated metadata to a reliable storage, so that | ||
| * they can be recovered after driver failures. See the Spark docs for more information on how | ||
| * to plug in your own custom implementation of a write ahead log. | ||
| */ | ||
| @org.apache.spark.annotation.DeveloperApi | ||
| public interface WriteAheadLog { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is the idea that this would be useful for Java implementations to keep this a Java interface?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. Its meant for users to create arbitrary implementations and we want to
|
||
| /** | ||
| * Write the record to the log and return the segment information that is necessary to read | ||
| * back the written record. The time is used to the index the record, such that it can be | ||
| * cleaned later. Note that the written data must be durable and readable (using the | ||
| * segment info) by the time this function returns. | ||
| */ | ||
| WriteAheadLogSegment write(ByteBuffer record, long time); | ||
|
|
||
| /** | ||
| * Read a written record based on the given segment information. | ||
| */ | ||
| ByteBuffer read(WriteAheadLogSegment segment); | ||
|
|
||
| /** | ||
| * Read and return an iterator of all the records that have written and not yet cleanup. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. not yet cleaned up. |
||
| */ | ||
| Iterator<ByteBuffer> readAll(); | ||
|
|
||
| /** | ||
| * Cleanup all the records that are older than the given threshold time. It can wait for | ||
| * the completion of the deletion. | ||
| */ | ||
| void cleanup(long threshTime, boolean waitForCompletion); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For some reason I feel like it should just be "clean" instead of "cleanup" (as in "log cleaning"). This is totally subjective though so I think the current one is OK to. |
||
|
|
||
| /** | ||
| * Close this log and release any resources. | ||
| */ | ||
| void close(); | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,26 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| /** | ||
| * This is an interface that represent the information required by any implementation of | ||
|
||
| * a WriteAheadLog to read a written record. | ||
|
||
| */ | ||
| @org.apache.spark.annotation.DeveloperApi | ||
| public interface WriteAheadLogSegment extends java.io.Serializable { | ||
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,14 +16,17 @@ | |
| */ | ||
| package org.apache.spark.streaming.rdd | ||
|
|
||
| import java.nio.ByteBuffer | ||
|
|
||
| import scala.reflect.ClassTag | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.commons.io.FileUtils | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.rdd.BlockRDD | ||
| import org.apache.spark.storage.{BlockId, StorageLevel} | ||
| import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader} | ||
| import org.apache.spark.streaming.util._ | ||
|
|
||
| /** | ||
| * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]]. | ||
|
|
@@ -37,7 +40,7 @@ private[streaming] | |
| class WriteAheadLogBackedBlockRDDPartition( | ||
| val index: Int, | ||
| val blockId: BlockId, | ||
| val segment: WriteAheadLogFileSegment) | ||
| val segment: WriteAheadLogSegment) | ||
| extends Partition | ||
|
|
||
|
|
||
|
|
@@ -58,7 +61,7 @@ private[streaming] | |
| class WriteAheadLogBackedBlockRDD[T: ClassTag]( | ||
| @transient sc: SparkContext, | ||
| @transient blockIds: Array[BlockId], | ||
| @transient segments: Array[WriteAheadLogFileSegment], | ||
| @transient segments: Array[WriteAheadLogSegment], | ||
| storeInBlockManager: Boolean, | ||
| storageLevel: StorageLevel) | ||
| extends BlockRDD[T](sc, blockIds) { | ||
|
|
@@ -96,9 +99,27 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( | |
| logDebug(s"Read partition data of $this from block manager, block $blockId") | ||
| iterator | ||
| case None => // Data not found in Block Manager, grab it from write ahead log file | ||
| val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf) | ||
| val dataRead = reader.read(partition.segment) | ||
| reader.close() | ||
| var dataRead: ByteBuffer = null | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I feel dirty seeing nulls in scala There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why allocate two (at least) objects when it is completely obvious that they are not going to be used. The null does not get exposed to anything outside the function, and hence is okay to have. If you look at rest of the Spark source code, we dont strictly adhere to Scala-way of doing things, rather balance code understandability (limit the levels of functional nesting) and efficiency (while loops instead of for when perf matters) with Scala styles. |
||
| var writeAheadLog: WriteAheadLog = null | ||
| try { | ||
| val dummyDirectory = FileUtils.getTempDirectoryPath() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why here need to use
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So the default WAL is file based so a log directory is needed for it to work. However, the log directory is really not needed reading a particular record. But to read a single record you have to create a FileBasedWriteAheadLog object, which needs a log directory. Hence I am providing a dummy directory for this. I know that this is a little awkward. This is the cost of defining a single interface for both writing and reading single records. Earlier there were two independent classes (WALWriter and WALRandomReader) that was used for these two purposes, which has different requirements. But since I am trying make single interface that can be used for all reading and writing, the log directory must be provided in the constructor of the default file-based WAL. This results in the awkwardness. I dont quite like it myself, but it may practically be okay as long as we ensure that the FileBasedWAL does not create unnecessary directories/files when only reading a single record. I can add a test to ensure that. |
||
| writeAheadLog = WriteAheadLogUtils.createLogForReceiver( | ||
| SparkEnv.get.conf, dummyDirectory, hadoopConf) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also IIUC here if the journal system if not hadoop based, hadoopConf may not be available.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hadoopConf is always available through the SparkContext. Irrespective of whether Hadoop file system is used, a Hadoop conf is created by the SparkContext which is passed on to this location. If the WAL is not the default FileBasedWAL, then this parameter is just ignored (see the method
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What I'm thinking is that do we need to have this parameter for the interface, can we hide this into file-based WAL implementations.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The log directory needs to be passed through the IMO that duplicates code everywhere and uglier that this dummy dir approach. And also, this does not handle |
||
| dataRead = writeAheadLog.read(partition.segment) | ||
| } catch { | ||
| case NonFatal(e) => | ||
| throw new SparkException( | ||
| s"Could not read data from write ahead log segment ${partition.segment}", e) | ||
| } finally { | ||
| if (writeAheadLog != null) { | ||
| writeAheadLog.close() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. May be reset writeAheadLog to null after close to avoid unexpected behavior :).
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
| } | ||
| } | ||
| if (dataRead == null) { | ||
| throw new SparkException( | ||
| s"Could not read data from write ahead log segment ${partition.segment}, " + | ||
| s"read returned null") | ||
| } | ||
| logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}") | ||
| if (storeInBlockManager) { | ||
| blockManager.putBytes(blockId, dataRead, storageLevel) | ||
|
|
@@ -117,8 +138,14 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( | |
| override def getPreferredLocations(split: Partition): Seq[String] = { | ||
| val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] | ||
| val blockLocations = getBlockIdLocations().get(partition.blockId) | ||
| blockLocations.getOrElse( | ||
| HdfsUtils.getFileSegmentLocations( | ||
| partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)) | ||
| blockLocations.getOrElse { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It might make sense to add location info to the WALRecordHandle interface itself. This way, systems that are not HDFS, but still benefit from preferred locations can use it.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's a good point. I wasnt super sure of whether it is a good idea to have it in the interface in this version. We can add it later and maintain binary compatibility as the RecordHandle is an abstract class. Also It is still a developer API s. For now, I am going to merge this in to unblock #5732 . |
||
| partition.segment match { | ||
| case fileSegment: FileBasedWriteAheadLogSegment => | ||
| HdfsUtils.getFileSegmentLocations( | ||
| fileSegment.path, fileSegment.offset, fileSegment.length, hadoopConfig) | ||
| case _ => | ||
| Seq.empty | ||
| } | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It might be good to give a high level description like "A WriteAheadLog is any storage service capable of persisting data binary data associated with a particular time and removing all data older than a certain time."
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good idea. Will do.