forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 3
Initial HDFS Readers and Writers implementation. #12
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
Merged
Merged
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
e4e6f93
Basic implementation of HDFS WAL.
harishreedharan f6d3a9d
Make most methods in writer and reader thread-safe.
harishreedharan c6ffa46
Add a method to close files
harishreedharan ea6f1b4
Add preconditions checks to ensure that the writer or reader has not …
harishreedharan faff282
Making FileSegment a case class
harishreedharan 46e03ab
Making all HDFS Wal classes private to streaming
harishreedharan a810b0a
Make close methods synchronized, to ensure close does not get called …
harishreedharan fc53eac
Make the Sequential reader an Iterator. Rename all classes. Mark all …
harishreedharan 9dfbd66
Make all variables private
harishreedharan d86a518
Minor changes to definitions of methods
harishreedharan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
19 changes: 19 additions & 0 deletions
19
streaming/src/main/scala/org/apache/spark/streaming/storage/FileSegment.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,19 @@ | ||
/* | ||
* 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.storage | ||
|
||
private[streaming] case class FileSegment (path: String, offset: Long, length: Int) |
63 changes: 63 additions & 0 deletions
63
streaming/src/main/scala/org/apache/spark/streaming/storage/HdfsUtils.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.storage | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream, Path} | ||
|
||
private[streaming] object HdfsUtils { | ||
|
||
def getOutputStream(path: String): FSDataOutputStream = { | ||
// HDFS is not thread-safe when getFileSystem is called, so synchronize on that | ||
|
||
val dfsPath = new Path(path) | ||
val conf = new Configuration() | ||
val dfs = | ||
this.synchronized { | ||
dfsPath.getFileSystem(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): FSDataInputStream = { | ||
val dfsPath = new Path(path) | ||
val dfs = this.synchronized { | ||
dfsPath.getFileSystem(new Configuration()) | ||
} | ||
val instream = dfs.open(dfsPath) | ||
instream | ||
} | ||
|
||
def checkState(state: Boolean, errorMsg: => String) { | ||
if(!state) { | ||
throw new IllegalStateException(errorMsg) | ||
} | ||
} | ||
|
||
} |
46 changes: 46 additions & 0 deletions
46
streaming/src/main/scala/org/apache/spark/streaming/storage/WriteAheadLogRandomReader.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* 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.storage | ||
|
||
import java.io.Closeable | ||
|
||
private[streaming] class WriteAheadLogRandomReader(path: String) extends Closeable { | ||
|
||
private val instream = HdfsUtils.getInputStream(path) | ||
private var closed = false | ||
|
||
def read(segment: FileSegment): Array[Byte] = synchronized { | ||
assertOpen() | ||
instream.seek(segment.offset) | ||
val nextLength = instream.readInt() | ||
HdfsUtils.checkState(nextLength == segment.length, | ||
"Expected message length to be " + segment.length + ", " + "but was " + nextLength) | ||
val buffer = new Array[Byte](nextLength) | ||
instream.readFully(buffer) | ||
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.") | ||
} | ||
} | ||
|
69 changes: 69 additions & 0 deletions
69
streaming/src/main/scala/org/apache/spark/streaming/storage/WriteAheadLogReader.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,69 @@ | ||
/* | ||
* 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.storage | ||
|
||
import java.io.Closeable | ||
|
||
private[streaming] class WriteAheadLogReader(path: String) | ||
extends Iterator[Array[Byte]] with Closeable { | ||
|
||
private val instream = HdfsUtils.getInputStream(path) | ||
private var closed = false | ||
private var nextItem: Option[Array[Byte]] = None | ||
|
||
override def hasNext: Boolean = synchronized { | ||
assertOpen() | ||
if (nextItem.isDefined) { // handle the case where hasNext is called without calling next | ||
true | ||
} else { | ||
val available = instream.available() | ||
if (available < 4) { // Length of next block (which is an Int = 4 bytes) of data is unavailable! | ||
false | ||
} | ||
val length = instream.readInt() | ||
if (instream.available() < length) { | ||
false | ||
} | ||
val buffer = new Array[Byte](length) | ||
instream.readFully(buffer) | ||
nextItem = Some(buffer) | ||
true | ||
} | ||
} | ||
|
||
override def next(): Array[Byte] = synchronized { | ||
// TODO: Possible error case where there are not enough bytes in the stream | ||
// TODO: How to handle that? | ||
val data = nextItem.getOrElse { | ||
throw new IllegalStateException("next called without calling hasNext or after hasNext " + | ||
"returned false") | ||
} | ||
nextItem = None // Ensure the next hasNext call loads new data. | ||
data | ||
} | ||
|
||
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.") | ||
} | ||
|
||
} |
47 changes: 47 additions & 0 deletions
47
streaming/src/main/scala/org/apache/spark/streaming/storage/WriteAheadLogWriter.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
/* | ||
* 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.storage | ||
|
||
import java.io.Closeable | ||
|
||
private[streaming] class WriteAheadLogWriter(path: String) extends Closeable { | ||
private val stream = HdfsUtils.getOutputStream(path) | ||
private var nextOffset = stream.getPos | ||
private var closed = false | ||
|
||
// Data is always written as: | ||
// - Length - Long | ||
// - Data - of length = Length | ||
def write(data: Array[Byte]): FileSegment = synchronized { | ||
assertOpen() | ||
val segment = new FileSegment(path, nextOffset, data.length) | ||
stream.writeInt(data.length) | ||
stream.write(data) | ||
stream.hflush() | ||
nextOffset = stream.getPos | ||
segment | ||
} | ||
|
||
override private[streaming] def close(): Unit = synchronized { | ||
closed = true | ||
stream.close() | ||
} | ||
|
||
private def assertOpen() { | ||
HdfsUtils.checkState(!closed, "Stream is closed. Create a new Writer to write to file.") | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Does this really need to be a separate function?
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.
Just to avoid the String concat cost. This accomplishes the same thing as Preconditions.checkArgument in Guava, but that being Java does not have pass by name.