Skip to content
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-18910][SQL]Resolve faile to use UDF that jar file in hdfs. #16324

Closed
wants to merge 1 commit into from

Conversation

shenh062326
Copy link
Contributor

@shenh062326 shenh062326 commented Dec 17, 2016

What changes were proposed in this pull request?

When I create a UDF that jar file in hdfs, I can't use the UDF.

spark-sql> create function trans_array as 'com.test.udf.TransArray' using jar 'hdfs://host1:9000/spark/dev/share/libs/spark-proxy-server-biz-service-impl-1.0.0.jar';
spark-sql> describe function trans_array;
Function: test_db.trans_array
Class: com.alipay.spark.proxy.server.biz.service.impl.udf.TransArray
Usage: N/A.
Time taken: 0.127 seconds, Fetched 3 row(s)
spark-sql> select trans_array(1, '\|', id, position) as (id0, position0) from test_spark limit 10;
Error in query: Undefined function: 'trans_array'. This function is neither a registered temporary function nor a permanent function registered in the database 'test_db'.; line 1 pos 7

The reason is when org.apache.spark.sql.internal.SessionState.FunctionResourceLoader.loadResource, the uri.toURL throw exception with " failed unknown protocol: hdfs"

def addJar(path: String): Unit = {
sparkSession.sparkContext.addJar(path)
val uri = new Path(path).toUri
val jarURL = if (uri.getScheme == null) {
// path is a local file path without a URL scheme
new File(path).toURI.toURL
} else {
// path is a URL with a scheme
{color:red}uri.toURL{color}
}
jarClassLoader.addURL(jarURL)
Thread.currentThread().setContextClassLoader(jarClassLoader)
}

I think we should setURLStreamHandlerFactory method on URL with an instance of FsUrlStreamHandlerFactory, just like:

static {
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory());
}

How was this patch tested?

I have test it in my cluster.

@shenh062326 shenh062326 changed the title Resolve faile to use UDF that jar file in hdfs. [SPARK-18910][SQL]Resolve faile to use UDF that jar file in hdfs. Dec 17, 2016
@@ -2373,6 +2373,7 @@ class SparkContext(config: SparkConf) extends Logging {
* various Spark features.
*/
object SparkContext extends Logging {
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SparkContext might not be the good place. object SharedState might be better.

In addition, you also need to add comments to explain why you did it before this line.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for you comment, I will add it to SharedState.

@gatorsmile
Copy link
Member

Please add the description in the JIRA to the PR description. FYI, you still can edit the description after you create the PR.

@SparkQA
Copy link

SparkQA commented Dec 17, 2016

Test build #70296 has finished for PR 16324 at commit a91b08f.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

This method can only be called once per JVM, so it is typically executed in a static block. This limitation means that if some other part of your program—perhaps a third-party component outside your control— sets a URLStreamHandlerFactory, you won’t be able to use this approach for reading data from Hadoop.

@gatorsmile
Copy link
Member

This might not be the right solution to do it, as explained above.

@shenh062326
Copy link
Contributor Author

Should we download the UDF jar from hdfs.

@gatorsmile
Copy link
Member

gatorsmile commented Dec 17, 2016

First, I am not sure whether we should support reading UDF jar from HDFS. cc @rxin

Second, if we want to support it, the best reviewers are @zsxwing @tdas They added the file HDFSMetadataLog.scala recently

@shenh062326
Copy link
Contributor Author

Currently,we can create a UDF with jar in HDFS, but failed to use it.
Spark driver won't download the jar from HDFS, it only add the path to the classLoader.
If we don't support reading UDF jar from HDFS, we should download the UDF jar.
I think support reading UDF jar from HDFS is better.

@rxin
Copy link
Contributor

rxin commented Dec 17, 2016

This is to allow using jars defined using HDFS-API, not just HDFS right? In that case it sounds like a good idea too ... but we need a test case for it.

@shenh062326
Copy link
Contributor Author

I‘m sorry, @rxin, I don't understand what you mean.

@rxin
Copy link
Contributor

rxin commented Dec 18, 2016

I was saying we need to create a test case for this change.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Feb 9, 2017

(gentle ping @shenh062326 )

@melin
Copy link

melin commented Mar 16, 2017

in spark 2.1.0
0: jdbc:hive2://localhost:10000> add jar hdfs:///user/datacompute/datacompute-udf-1.0-
Error: java.net.MalformedURLException: unknown protocol: hdfs (state=,code=0)
0: jdbc:hive2://localhost:10000>

ghost pushed a commit to dbtsai/spark that referenced this pull request Apr 26, 2017
## What changes were proposed in this pull request?
Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](apache#16324) , but all of them are inactivity for a long time or have been closed.

This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate
UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler.

## How was this patch tested?
1. Add a new unit test.
2. Check manually.
Before: throw an exception with " failed unknown protocol: hdfs"
<img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png">

After:
<img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png">

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes apache#17342 from weiqingy/SPARK-18910.
asfgit pushed a commit that referenced this pull request Apr 26, 2017
## What changes were proposed in this pull request?
Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](#16324) , but all of them are inactivity for a long time or have been closed.

This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate
UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler.

## How was this patch tested?
1. Add a new unit test.
2. Check manually.
Before: throw an exception with " failed unknown protocol: hdfs"
<img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png">

After:
<img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png">

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #17342 from weiqingy/SPARK-18910.

(cherry picked from commit 2ba1eba)
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
6 participants