Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
Expand Up @@ -21,7 +21,7 @@ import org.apache.spark.unsafe.types.UTF8String

/**
* This holds file names of the current Spark task. This is used in HadoopRDD,
* FileScanRDD and InputFileName function in Spark SQL.
* FileScanRDD, NewHadoopRDD and InputFileName function in Spark SQL.
*/
private[spark] object InputFileNameHolder {
/**
Expand Down
7 changes: 7 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,12 @@ class NewHadoopRDD[K, V](
val inputMetrics = context.taskMetrics().inputMetrics
val existingBytesRead = inputMetrics.bytesRead

// Sets the thread local variable for the file's name
split.serializableHadoopSplit.value match {
case fs: FileSplit => InputFileNameHolder.setInputFileName(fs.getPath.toString)
case _ => InputFileNameHolder.unsetInputFileName()
}

// Find a function that will return the FileSystem bytes read by this thread. Do this before
// creating RecordReader, because RecordReader's constructor might read some bytes
val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match {
Expand Down Expand Up @@ -201,6 +207,7 @@ class NewHadoopRDD[K, V](

private def close() {
if (reader != null) {
InputFileNameHolder.unsetInputFileName()
// Close the reader and release it. Note: it's very important that we don't close the
// reader more than once, since that exposes us to MAPREDUCE-5918 when running against
// Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql

import org.apache.hadoop.io.{LongWritable, Text}
import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat}
import org.scalatest.Matchers._

import org.apache.spark.sql.catalyst.expressions.NamedExpression
Expand Down Expand Up @@ -592,7 +594,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext {
)
}

test("input_file_name") {
test("input_file_name - FileScanRDD") {
withTempPath { dir =>
val data = sparkContext.parallelize(0 to 10).toDF("id")
data.write.parquet(dir.getCanonicalPath)
Expand All @@ -604,6 +606,35 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext {
}
}

test("input_file_name - HadoopRDD") {
withTempPath { dir =>
val data = sparkContext.parallelize((0 to 10).map(_.toString)).toDF()
data.write.text(dir.getCanonicalPath)
val df = spark.sparkContext.textFile(dir.getCanonicalPath).toDF()
val answer = df.select(input_file_name()).head.getString(0)
assert(answer.contains(dir.getCanonicalPath))

checkAnswer(data.select(input_file_name()).limit(1), Row(""))
}
}

test("input_file_name - NewHadoopRDD") {
withTempPath { dir =>
val data = sparkContext.parallelize((0 to 10).map(_.toString)).toDF()
data.write.text(dir.getCanonicalPath)
val rdd = spark.sparkContext.newAPIHadoopFile(
dir.getCanonicalPath,
classOf[NewTextInputFormat],
classOf[LongWritable],
classOf[Text])
val df = rdd.map(pair => pair._2.toString).toDF()
val answer = df.select(input_file_name()).head.getString(0)
assert(answer.contains(dir.getCanonicalPath))

checkAnswer(data.select(input_file_name()).limit(1), Row(""))
}
}

test("columns can be compared") {
assert('key.desc == 'key.desc)
assert('key.desc != 'key.asc)
Expand Down Expand Up @@ -707,5 +738,4 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext {
testData2.select($"a".bitwiseXOR($"b").bitwiseXOR(39)),
testData2.collect().toSeq.map(r => Row(r.getInt(0) ^ r.getInt(1) ^ 39)))
}

}