Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-11447

Null comparison requires type information but type extraction fails for complex types

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.5.1
    • 1.6.0
    • SQL
    • None

    Description

      While comparing a Column to a null literal, comparison works only if type of null literal matches type of the Column it's being compared to. Example scala code (can be run from spark shell):

      import org.apache.spark.sql._
      import org.apache.spark.sql.types._
      import org.apache.spark.sql.catalyst.expressions._

      val inputRowsData = Seq(Seq("abc"),Seq(null),Seq("xyz"))
      val inputRows = for(seq <- inputRowsData) yield Row.fromSeq(seq)
      val dfSchema = StructType(Seq(StructField("column", StringType, true)))
      val df = sqlContext.createDataFrame(sc.makeRDD(inputRows), dfSchema)

      //DOESN'T WORK
      val filteredDF = df.filter(df("column") <=> (new Column(Literal(null))))

      //WORKS
      val filteredDF = df.filter(df("column") <=> (new Column(Literal.create(null, SparkleFunctions.dataType(df("column"))))))

      Why should type information be required for a null comparison? If it's required, it's not always possible to extract type information from complex types (e.g. StructType). Following scala code (can be run from spark shell), throws org.apache.spark.sql.catalyst.analysis.UnresolvedException:

      import org.apache.spark.sql._
      import org.apache.spark.sql.types._
      import org.apache.spark.sql.catalyst.expressions._

      val inputRowsData = Seq(Seq(Row.fromSeq(Seq("abc", "def"))),Seq(Row.fromSeq(Seq(null, "123"))),Seq(Row.fromSeq(Seq("ghi", "jkl"))))
      val inputRows = for(seq <- inputRowsData) yield Row.fromSeq(seq)
      val dfSchema = StructType(Seq(StructField("column", StructType(Seq(StructField("p1", StringType, true), StructField("p2", StringType, true))), true)))

      val filteredDF = df.filter(df("column")("p1") <=> (new Column(Literal.create(null, SparkleFunctions.dataType(df("column")("p1"))))))

      org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: column#0[p1]
      at org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue.dataType(unresolved.scala:243)
      at org.apache.spark.sql.ArithmeticFunctions$class.dataType(ArithmeticFunctions.scala:76)
      at org.apache.spark.sql.SparkleFunctions$.dataType(SparkleFunctions.scala:14)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:38)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:43)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:45)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:47)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:49)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:51)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:53)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:55)
      at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:57)
      at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:59)
      at $iwC$$iwC$$iwC$$iwC.<init>(<console>:61)
      at $iwC$$iwC$$iwC.<init>(<console>:63)
      at $iwC$$iwC.<init>(<console>:65)
      at $iwC.<init>(<console>:67)
      at <init>(<console>:69)
      at .<init>(<console>:73)
      at .<clinit>(<console>)
      at .<init>(<console>:7)
      at .<clinit>(<console>)
      at $print(<console>)
      at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      at java.lang.reflect.Method.invoke(Method.java:606)
      at org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
      at org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1340)
      at org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
      at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)
      at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)
      at org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857)
      at org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902)
      at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814)
      at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657)
      at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665)
      at org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670)
      at org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997)
      at org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
      at org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
      at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
      at org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945)
      at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059)
      at org.apache.spark.repl.Main$.main(Main.scala:31)
      at org.apache.spark.repl.Main.main(Main.scala)
      at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      at java.lang.reflect.Method.invoke(Method.java:606)
      at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:672)
      at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
      at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
      at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120)
      at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

      Attachments

        Activity

          People

            kevinyu98 kevin yu
            kapilsingh5050 Kapil Singh
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: