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 @@ -41,9 +41,10 @@ private[spark] object InputFileBlockHolder {
* The thread variable for the name of the current file being read. This is used by
* the InputFileName function in Spark SQL.
*/
private[this] val inputBlock: ThreadLocal[FileBlock] = new ThreadLocal[FileBlock] {
override protected def initialValue(): FileBlock = new FileBlock
}
private[this] val inputBlock: InheritableThreadLocal[FileBlock] =
new InheritableThreadLocal[FileBlock] {
override protected def initialValue(): FileBlock = new FileBlock
}

/**
* Returns the holding file name or empty string if it is unknown.
Expand Down
24 changes: 24 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -435,6 +435,30 @@ def test_udf_with_input_file_name(self):
row = self.spark.read.json(filePath).select(sourceFile(input_file_name())).first()
self.assertTrue(row[0].find("people1.json") != -1)

def test_udf_with_input_file_name_for_hadooprdd(self):
from pyspark.sql.functions import udf, input_file_name
from pyspark.sql.types import StringType

def filename(path):
return path

sameText = udf(filename, StringType())

rdd = self.sc.textFile('python/test_support/sql/people.json')
df = self.spark.read.json(rdd).select(input_file_name().alias('file'))
row = df.select(sameText(df['file'])).first()
self.assertTrue(row[0].find("people.json") != -1)

rdd2 = self.sc.newAPIHadoopFile(
'python/test_support/sql/people.json',
'org.apache.hadoop.mapreduce.lib.input.TextInputFormat',
'org.apache.hadoop.io.LongWritable',
'org.apache.hadoop.io.Text')

df2 = self.spark.read.json(rdd2).select(input_file_name().alias('file'))
row2 = df2.select(sameText(df2['file'])).first()
self.assertTrue(row2[0].find("people.json") != -1)

def test_basic_functions(self):
rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}'])
df = self.spark.read.json(rdd)
Expand Down