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-5741][SQL] Support the path contains comma in HiveContext #4532

Closed
wants to merge 3 commits into from

Conversation

watermen
Copy link
Contributor

When run select * from nzhang_part where hr = 'file,';, it throws exception java.lang.IllegalArgumentException: Can not create a Path from an empty string
. Because the path of hdfs contains comma, and FileInputFormat.setInputPaths will split path by comma.

SQL

set hive.merge.mapfiles=true; 
set hive.merge.mapredfiles=true;
set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
set hive.exec.dynamic.partition=true;
set hive.exec.dynamic.partition.mode=nonstrict;

create table nzhang_part like srcpart;

insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08';

insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08';

insert overwrite table nzhang_part partition (ds='2010-08-15', hr) 
select * from (
select key, value, hr from srcpart where ds='2008-04-08'
union all
select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;

select * from nzhang_part where hr = 'file,';

Error Log

15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part where hr = 'file,']
java.lang.IllegalArgumentException: Can not create a Path from an empty string
at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
at org.apache.hadoop.fs.Path.<init>(Path.java:135)
at org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at scala.Option.map(Option.scala:145)
at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@watermen watermen changed the title [SPARK-5741][SQL] Support comma in path in HiveContext [SPARK-5741][SQL] Support the path contains comma in HiveContext Feb 11, 2015
@adrian-wang
Copy link
Contributor

ok to test.

@@ -248,7 +249,7 @@ private[hive] object HadoopTableReader extends HiveInspectors {
* instantiate a HadoopRDD.
*/
def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) {
FileInputFormat.setInputPaths(jobConf, path)
jobConf.set("mapred.input.dir", StringUtils.escapeString(path.toString()))
Copy link
Contributor

Choose a reason for hiding this comment

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

Instead of setting the conf using the key, can we still use FileInputFormat.setInputPaths? Like

FileInputFormat.setInputPaths(jobConf, StringUtils.escapeString(path))

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Can't, for examples "hdfs://x.x.x.x:9000/user/hive/warehouse/nzhang_part/ds=2010-08-15/hr=file," is will be splited into "hdfs://x.x.x.x:9000/user/hive/warehouse/nzhang_part/ds=2010-08-15/hr=file" and "" by FileInputFormat.getPathStrings, "" will be checked by Path.checkPathArg and

if( path.length() == 0 ) {
    throw new IllegalArgumentException("Can not create a Path from an empty string");
}

you can see

FileInputFormat.setInputPaths -> FileInputFormat.getPathStrings -> Path.checkPathArg

in hadoop for detail.

Copy link
Contributor

Choose a reason for hiding this comment

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

o, I see. getPathStrings does not really care if a comma is escaped or not... Can we use public static void setInputPaths(Job job, Path... inputPaths)? I think it is better to avoid using set directly with a string key (using a method seems more robust).

@yhuai
Copy link
Contributor

yhuai commented Feb 11, 2015

ok to test

@SparkQA
Copy link

SparkQA commented Feb 11, 2015

Test build #27292 has started for PR 4532 at commit b788a72.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 11, 2015

Test build #27292 has finished for PR 4532 at commit b788a72.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27292/
Test PASSed.

@scwf
Copy link
Contributor

scwf commented Feb 12, 2015

lgtm

@watermen
Copy link
Contributor Author

@yhuai Can you review it?

@SparkQA
Copy link

SparkQA commented Feb 27, 2015

Test build #28034 has started for PR 4532 at commit 9758ab1.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 27, 2015

Test build #28034 has finished for PR 4532 at commit 9758ab1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28034/
Test PASSed.

@watermen
Copy link
Contributor Author

@yhuai Can you review the code for me?

@yhuai
Copy link
Contributor

yhuai commented Feb 27, 2015

LGTM

@watermen
Copy link
Contributor Author

watermen commented Mar 2, 2015

@marmbrus @rxin Can it be merged?

@marmbrus
Copy link
Contributor

marmbrus commented Mar 2, 2015

Thanks! Merging to master and 1.3.

asfgit pushed a commit that referenced this pull request Mar 2, 2015
When run ```select * from nzhang_part where hr = 'file,';```, it throws exception ```java.lang.IllegalArgumentException: Can not create a Path from an empty string```
. Because the path of hdfs contains comma, and FileInputFormat.setInputPaths will split path by comma.

### SQL
```
set hive.merge.mapfiles=true;
set hive.merge.mapredfiles=true;
set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
set hive.exec.dynamic.partition=true;
set hive.exec.dynamic.partition.mode=nonstrict;

create table nzhang_part like srcpart;

insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08';

insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08';

insert overwrite table nzhang_part partition (ds='2010-08-15', hr)
select * from (
select key, value, hr from srcpart where ds='2008-04-08'
union all
select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;

select * from nzhang_part where hr = 'file,';
```

### Error Log
```
15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part where hr = 'file,']
java.lang.IllegalArgumentException: Can not create a Path from an empty string
at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
at org.apache.hadoop.fs.Path.<init>(Path.java:135)
at org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at scala.Option.map(Option.scala:145)
at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)

Author: q00251598 <qiyadong@huawei.com>

Closes #4532 from watermen/SPARK-5741 and squashes the following commits:

9758ab1 [q00251598] fix bug
1db1a1c [q00251598] use setInputPaths(Job job, Path... inputPaths)
b788a72 [q00251598] change FileInputFormat.setInputPaths to jobConf.set and add test suite

(cherry picked from commit 9ce12aa)
Signed-off-by: Michael Armbrust <michael@databricks.com>
@asfgit asfgit closed this in 9ce12aa Mar 2, 2015
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants