forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 10
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge remote-tracking branch 'upstream/master' into spark-1403
- Loading branch information
Showing
11 changed files
with
328 additions
and
12 deletions.
There are no files selected for viewing
This file contains 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
This file contains 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
This file contains 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
This file contains 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
This file contains 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
This file contains 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
47 changes: 47 additions & 0 deletions
47
core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
This file contains 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.input | ||
|
||
import org.apache.hadoop.fs.Path | ||
import org.apache.hadoop.mapreduce.InputSplit | ||
import org.apache.hadoop.mapreduce.JobContext | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat | ||
import org.apache.hadoop.mapreduce.RecordReader | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit | ||
|
||
/** | ||
* A [[org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat CombineFileInputFormat]] for | ||
* reading whole text files. Each file is read as key-value pair, where the key is the file path and | ||
* the value is the entire content of file. | ||
*/ | ||
|
||
private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[String, String] { | ||
override protected def isSplitable(context: JobContext, file: Path): Boolean = false | ||
|
||
override def createRecordReader( | ||
split: InputSplit, | ||
context: TaskAttemptContext): RecordReader[String, String] = { | ||
|
||
new CombineFileRecordReader[String, String]( | ||
split.asInstanceOf[CombineFileSplit], | ||
context, | ||
classOf[WholeTextFileRecordReader]) | ||
} | ||
} |
72 changes: 72 additions & 0 deletions
72
core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala
This file contains 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,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.input | ||
|
||
import com.google.common.io.{ByteStreams, Closeables} | ||
|
||
import org.apache.hadoop.io.Text | ||
import org.apache.hadoop.mapreduce.InputSplit | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit | ||
import org.apache.hadoop.mapreduce.RecordReader | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
|
||
/** | ||
* A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file | ||
* out in a key-value pair, where the key is the file path and the value is the entire content of | ||
* the file. | ||
*/ | ||
private[spark] class WholeTextFileRecordReader( | ||
split: CombineFileSplit, | ||
context: TaskAttemptContext, | ||
index: Integer) | ||
extends RecordReader[String, String] { | ||
|
||
private val path = split.getPath(index) | ||
private val fs = path.getFileSystem(context.getConfiguration) | ||
|
||
// True means the current file has been processed, then skip it. | ||
private var processed = false | ||
|
||
private val key = path.toString | ||
private var value: String = null | ||
|
||
override def initialize(split: InputSplit, context: TaskAttemptContext) = {} | ||
|
||
override def close() = {} | ||
|
||
override def getProgress = if (processed) 1.0f else 0.0f | ||
|
||
override def getCurrentKey = key | ||
|
||
override def getCurrentValue = value | ||
|
||
override def nextKeyValue = { | ||
if (!processed) { | ||
val fileIn = fs.open(path) | ||
val innerBuffer = ByteStreams.toByteArray(fileIn) | ||
|
||
value = new Text(innerBuffer).toString | ||
Closeables.close(fileIn, false) | ||
|
||
processed = true | ||
true | ||
} else { | ||
false | ||
} | ||
} | ||
} |
This file contains 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
Oops, something went wrong.