You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Seydou Dia (JIRA)" <ji...@apache.org> on 2017/07/06 16:59:00 UTC

[jira] [Commented] (SPARK-21227) Unicode in Json field causes AnalysisException when selecting from Dataframe

    [ https://issues.apache.org/jira/browse/SPARK-21227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16076897#comment-16076897 ] 

Seydou Dia commented on SPARK-21227:
------------------------------------

Hi [~srowen],

I had time to dig further the issue and come up with a solution. As [~hyukjin.kwon] first pointed out, the problem was the use of the method 
{code:java} equalsIgnoreCase {code}. In the Java implementation the strings to be compared are changed to uppercase, hence i and dotless  evaluated equal. The solution I suggest is to switch to {code:java}toLowerCase{code} based comparison. None of the previous methods rely on Locals().

Please find below my diff already tested on my local machine:

{code:diff}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
index 7731336..d6e1e9a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
    b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
@@ -33,7  33,7 @@ package object analysis {
    */
   type Resolver = (String, String) => Boolean
 
-  val caseInsensitiveResolution = (a: String, b: String) => a.equalsIgnoreCase(b)
   val caseInsensitiveResolution = (a: String, b: String) => a.toLowerCase() == b.toLowerCase()
   val caseSensitiveResolution = (a: String, b: String) => a == b
 
   implicit class AnalysisErrorAt(t: TreeNode[_]) {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
index 7311dc3..aae108c 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
    b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
@@ -579,4  579,23 @@ class AnalysisErrorSuite extends AnalysisTest {
     assertAnalysisError(plan5,
                         "Accessing outer query column is not allowed in" :: Nil)
   }
 
 
   test("SPARK-21227: i and dotless i should not be considered the same characters in column name.") {
       // scalastyle:off nonascii
       val relation = LocalRelation(
           AttributeReference("city_name", StringType)(),
           AttributeReference("c\u0131ty_name", StringType)()
       )
 
       val plan1 = relation.select($"city_name")
       assertAnalysisSuccess(plan1, caseSensitive = false)
       assertAnalysisSuccess(plan1, caseSensitive = true)
 
       val plan2 = relation.select($"c\u0131ty_name")
       assertAnalysisSuccess(plan2, caseSensitive = false)
       assertAnalysisSuccess(plan2, caseSensitive = true)
       // scalastyle:on nonascii
   }
 
 }


{code}




Ref:
http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/6-b14/java/lang/String.java#String.equalsIgnoreCase%28java.lang.String%29


> Unicode in Json field causes AnalysisException when selecting from Dataframe
> ----------------------------------------------------------------------------
>
>                 Key: SPARK-21227
>                 URL: https://issues.apache.org/jira/browse/SPARK-21227
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark
>    Affects Versions: 2.1.0
>            Reporter: Seydou Dia
>
> Hi,
> please find below the step to reproduce the issue I am facing.
> First I create a json with 2 fields:
> * city_name
> * cıty_name
> The first one is valid ascii, while the second contains a unicode (ı, i without dot ).
> When I try to select from the dataframe I have an  {noformat} AnalysisException {noformat}.
> {code:python}
> $ pyspark
> Python 3.4.3 (default, Sep  1 2016, 23:33:38) 
> [GCC 4.8.3 20140911 (Red Hat 4.8.3-9)] on linux
> Type "help", "copyright", "credits" or "license" for more information.
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
> 17/06/27 12:29:05 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041.
> 17/06/27 12:29:05 WARN Utils: Service 'SparkUI' could not bind on port 4041. Attempting port 4042.
> 17/06/27 12:29:08 WARN Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME.
> Welcome to
>       ____              __
>      / __/__  ___ _____/ /__
>     _\ \/ _ \/ _ `/ __/  '_/
>    /__ / .__/\_,_/_/ /_/\_\   version 2.1.0
>       /_/
> Using Python version 3.4.3 (default, Sep  1 2016 23:33:38)
> SparkSession available as 'spark'.
> >>> sc=spark.sparkContext
> >>> js = ['{"city_name": "paris"}'
> ...     , '{"city_name": "rome"}'
> ...     , '{"city_name": "berlin"}'
> ...     , '{"cıty_name": "new-york"}'
> ...     , '{"cıty_name": "toronto"}'
> ...     , '{"cıty_name": "chicago"}'
> ...     , '{"cıty_name": "dubai"}']
> >>> myRDD = sc.parallelize(js)
> >>> myDF = spark.read.json(myRDD)
> >>> myDF.printSchema()                                                          
> root
>  |-- city_name: string (nullable = true)
>  |-- cıty_name: string (nullable = true)
> >>> myDF.select(myDF['city_name'])
> Traceback (most recent call last):
>   File "/usr/lib/spark/python/pyspark/sql/utils.py", line 63, in deco
>     return f(*a, **kw)
>   File "/usr/lib/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", line 319, in get_return_value
> py4j.protocol.Py4JJavaError: An error occurred while calling o226.apply.
> : org.apache.spark.sql.AnalysisException: Reference 'city_name' is ambiguous, could be: city_name#29, city_name#30.;
> 	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:264)
> 	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:168)
> 	at org.apache.spark.sql.Dataset.resolve(Dataset.scala:217)
> 	at org.apache.spark.sql.Dataset.col(Dataset.scala:1073)
> 	at org.apache.spark.sql.Dataset.apply(Dataset.scala:1059)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:498)
> 	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
> 	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
> 	at py4j.Gateway.invoke(Gateway.java:280)
> 	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
> 	at py4j.commands.CallCommand.execute(CallCommand.java:79)
> 	at py4j.GatewayConnection.run(GatewayConnection.java:214)
> 	at java.lang.Thread.run(Thread.java:745)
> During handling of the above exception, another exception occurred:
> Traceback (most recent call last):
>   File "<stdin>", line 1, in <module>
>   File "/usr/lib/spark/python/pyspark/sql/dataframe.py", line 943, in __getitem__
>     jc = self._jdf.apply(item)
>   File "/usr/lib/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 1133, in __call__
>   File "/usr/lib/spark/python/pyspark/sql/utils.py", line 69, in deco
>     raise AnalysisException(s.split(': ', 1)[1], stackTrace)
> pyspark.sql.utils.AnalysisException: "Reference 'city_name' is ambiguous, could be: city_name#29, city_name#30.;"
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org