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
[SPARK-19223][SQL][PySpark] Fix InputFileBlockHolder for datasources which are based on HadoopRDD or NewHadoopRDD #16585
Conversation
Test build #71366 has started for PR 16585 at commit |
retest this please. |
Test build #71368 has finished for PR 16585 at commit
|
retest this please. |
Test build #71370 has finished for PR 16585 at commit
|
ping @rxin @cloud-fan |
should the proper fix be the python thread transfers the proper information over? |
@rxin Thanks for looking at this. I think the simplest way to transfer the info is using |
SGTM |
@cloud-fan SGTM is for current approach or |
|
e2d872c
to
1563e03
Compare
BTW please add a test case for this. Thanks. |
Test build #71474 has finished for PR 16585 at commit
|
Test build #71475 has finished for PR 16585 at commit
|
157f70f
to
d7c05d2
Compare
import org.apache.spark.{SparkContext, SparkFunSuite} | ||
import org.apache.spark.util.Utils | ||
|
||
class InputFileBlockHolderSuite extends SparkFunSuite { |
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.
shall we just write a pyspark test?
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.
I can't find a proper built-in datasource for testing it. I think all file-based datasources are based on FileFormat
. They don't use HadoopRDD/NewHadoopRDD. The pyspark test shown in the description is using spark-xml package to test manually.
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.
I've written a pyspark test which directly reads file and produces HadoopRDD/NewHadoopRDD based dataframe. So this scala test is removed.
Test build #71511 has finished for PR 16585 at commit
|
Test build #71509 has finished for PR 16585 at commit
|
Test build #71510 has finished for PR 16585 at commit
|
d7c05d2
to
8380617
Compare
Test build #71540 has finished for PR 16585 at commit
|
Test build #71559 has finished for PR 16585 at commit
|
3444499
to
2ce65cb
Compare
Test build #71561 has finished for PR 16585 at commit
|
def filename(path): | ||
return path | ||
|
||
self.spark.udf.register('sameText', filename) |
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.
where do we call this registered 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.
oh. wrongly copied.
'org.apache.hadoop.io.Text') | ||
|
||
df2 = self.spark.read.json(rdd2).select(input_file_name().alias('file')) | ||
row = df2.select(sameText(df2['file'])).first() |
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.
nit: row2
?
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.
sure.
Test build #71575 has started for PR 16585 at commit |
LGTM, pending jenkins |
retest this please. |
Test build #71581 has finished for PR 16585 at commit
|
retest this please. |
Test build #71596 has finished for PR 16585 at commit
|
thanks, merging to master! |
@rxin @cloud-fan Thanks! |
…which are based on HadoopRDD or NewHadoopRDD ## What changes were proposed in this pull request? For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF. The method to reproduce it is, running the following codes with `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`: from pyspark.sql.functions import udf,input_file_name from pyspark.sql.types import StringType from pyspark.sql import SparkSession def filename(path): return path session = SparkSession.builder.appName('APP').getOrCreate() session.udf.register('sameText', filename) sameText = udf(filename, StringType()) df = session.read.format('xml').load('a.xml', rowTag='root').select('*', input_file_name().alias('file')) df.select('file').show() # works df.select(sameText(df['file'])).show() # returns empty content The issue is because in `HadoopRDD` and `NewHadoopRDD` we set the file block's info in `InputFileBlockHolder` before the returned iterator begins consuming. `InputFileBlockHolder` will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread. To fix this, we have to set the info in `InputFileBlockHolder` after the iterator begins consuming. So the info can be read in correct thread. ## How was this patch tested? Manual test with above example codes for spark-xml package on pyspark: `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`. Added pyspark test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes apache#16585 from viirya/fix-inputfileblock-hadooprdd.
…which are based on HadoopRDD or NewHadoopRDD ## What changes were proposed in this pull request? For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF. The method to reproduce it is, running the following codes with `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`: from pyspark.sql.functions import udf,input_file_name from pyspark.sql.types import StringType from pyspark.sql import SparkSession def filename(path): return path session = SparkSession.builder.appName('APP').getOrCreate() session.udf.register('sameText', filename) sameText = udf(filename, StringType()) df = session.read.format('xml').load('a.xml', rowTag='root').select('*', input_file_name().alias('file')) df.select('file').show() # works df.select(sameText(df['file'])).show() # returns empty content The issue is because in `HadoopRDD` and `NewHadoopRDD` we set the file block's info in `InputFileBlockHolder` before the returned iterator begins consuming. `InputFileBlockHolder` will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread. To fix this, we have to set the info in `InputFileBlockHolder` after the iterator begins consuming. So the info can be read in correct thread. ## How was this patch tested? Manual test with above example codes for spark-xml package on pyspark: `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`. Added pyspark test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes apache#16585 from viirya/fix-inputfileblock-hadooprdd.
What changes were proposed in this pull request?
For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF.
The method to reproduce it is, running the following codes with
bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1
:The issue is because in
HadoopRDD
andNewHadoopRDD
we set the file block's info inInputFileBlockHolder
before the returned iterator begins consuming.InputFileBlockHolder
will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread.To fix this, we have to set the info in
InputFileBlockHolder
after the iterator begins consuming. So the info can be read in correct thread.How was this patch tested?
Manual test with above example codes for spark-xml package on pyspark:
bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1
.Added pyspark test.
Please review http://spark.apache.org/contributing.html before opening a pull request.