-
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
Changes from 6 commits
e4e6f93
f6d3a9d
c6ffa46
ea6f1b4
faff282
46e03ab
a810b0a
fc53eac
9dfbd66
d86a518
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,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) |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| /* | ||
| * 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] class HdfsSequentialReader(val path: String) { | ||
|
|
||
| val instream = HdfsUtils.getInputStream(path) | ||
| var closed = false | ||
|
|
||
| def hasNext: Boolean = { | ||
| assertOpen() | ||
| synchronized { | ||
| instream.available() != 0 | ||
| } | ||
| } | ||
|
|
||
| def readNext(): Array[Byte] = { | ||
| assertOpen() | ||
| // TODO: Possible error case where there are not enough bytes in the stream | ||
|
||
| // TODO: How to handle that? | ||
| synchronized { | ||
| val length = instream.readInt() | ||
| HdfsUtils.checkState(length <= instream.available(), "Not enough data found in file!") | ||
| val buffer = new Array[Byte](length) | ||
| instream.readFully(buffer) | ||
| buffer | ||
| } | ||
| } | ||
|
|
||
| def close() { | ||
| closed = true | ||
| instream.close() | ||
| } | ||
|
|
||
| def assertOpen() { | ||
|
||
| HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the " + | ||
| "file.") | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,60 @@ | ||
| /* | ||
| * 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.{FileStatus, 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(new Configuration()) | ||
|
Owner
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 not use the conf defined above?
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. Fixed. |
||
| } | ||
| // If the file exists and we have append support, append instead of creating a new file | ||
| val stream: FSDataOutputStream = { | ||
| if (conf.getBoolean("hdfs.append.support", false) && dfs.isFile(dfsPath)) { | ||
|
Owner
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 do you ever have to append? Isnt it fine to create new files? We are have to create new files at some not-so-frequent intervals (say 1 minute), because we also need to delete old files, containing data that is not needed any more.
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 idea is to be as generic as possible here. If the file already exists, the create call will fail. Since the file name is decided a layer above this one, whatever class is writing to HDFS can decide whether to append to a file or to create a new one. If in some case, we create new instances of this class with the same file name, we can still write data, just appending to an existing file. I don't see anything wrong with having this supported - if it is used, great else this code path is not exercised at all.
Owner
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. But what happens when the file exists and append support isnt present? It will throw an error, right? And that may very well happen on some deployments. So any ways, all modules that use this have to any way be written such that filenames are unique, so that it does not fail on certain systems. Isnt it? I dont mind the code, since it is a small piece of code. Just that it doesnt really help us in any way and is therefore redundant.
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. Since hadoop-.20 or so append is supported by default. I have never come across a situation where append has not been enabled since like 2010. So I don't see that as a major concern. If needed, we can resolve it later. |
||
| dfs.append(dfsPath) | ||
| } else { | ||
| dfs.create(dfsPath) | ||
| } | ||
| } | ||
| stream | ||
| } | ||
|
|
||
| def getInputStream(path: String): FSDataInputStream = { | ||
| val dfsPath = new Path(path) | ||
| val conf = new Configuration() | ||
|
Owner
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. conf not used anywhere.
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. Fixed |
||
| val dfs = this.synchronized { | ||
| dfsPath.getFileSystem(new Configuration()) | ||
| } | ||
| val instream = dfs.open(dfsPath) | ||
| instream | ||
| } | ||
|
|
||
| def checkState(state: Boolean, errorMsg: => String) { | ||
|
Owner
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. Does this really need to be a separate function?
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. 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. |
||
| if(!state) { | ||
| throw new IllegalStateException(errorMsg) | ||
| } | ||
| } | ||
|
|
||
| } | ||
| 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 | ||
|
|
||
| private[streaming] class HdfsWalRandomReader(val path: String) { | ||
|
|
||
| val instream = HdfsUtils.getInputStream(path) | ||
| var closed = false | ||
|
|
||
| def read(segment: FileSegment): Array[Byte] = { | ||
| assertOpen() | ||
| synchronized { | ||
| 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 | ||
| } | ||
| } | ||
|
|
||
| def close() { | ||
| closed = true | ||
| instream.close() | ||
| } | ||
|
|
||
| def assertOpen() { | ||
| HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the file.") | ||
| } | ||
| } | ||
|
|
| 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 | ||
|
|
||
| private[streaming] class HdfsWalWriter(val path: String) { | ||
| val stream = HdfsUtils.getOutputStream(path) | ||
| var nextOffset = stream.getPos | ||
| var closed = false | ||
|
|
||
| // Data is always written as: | ||
| // - Length - Long | ||
| // - Data - of length = Length | ||
| def write(data: Array[Byte]): FileSegment = { | ||
| assertOpen() | ||
| synchronized { | ||
| val segment = new FileSegment(path, nextOffset, data.length) | ||
| stream.writeInt(data.length) | ||
| stream.write(data) | ||
| stream.hflush() | ||
| nextOffset = stream.getPos | ||
| segment | ||
| } | ||
| } | ||
|
|
||
| def close(): Unit = { | ||
| closed = true | ||
| stream.close() | ||
| } | ||
|
|
||
| def assertOpen() { | ||
|
||
| HdfsUtils.checkState(!closed, "Stream is closed. Create a new Writer to write to file.") | ||
| } | ||
| } | ||
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.
Lets name it WriteAheadLogReader. And the other one WriteAheadLogWriter. It does not need to be specific to HDFS.
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.
and WriteAheadLogRandomReader?