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

Scala 2.12 issues: Compilation error with sbt

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.4.0
    • 2.4.0
    • SQL
    • None

    Description

      When compiling with sbt, the following errors occur:

      There are two three types:
      1. ExprValue.isNull is compared with unexpected type.
      2. match may not be exhaustive is detected at match
      3. discarding unmoored doc comment

      The first one is more serious since it may also generate incorrect code in Spark 2.3.

      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala:63: match may not be exhaustive.
      [error] It would fail on the following inputs: (NumericValueInterval(_, _), _), (_, NumericValueInterval(_, _)), (_, _)
      [error] [warn]   def isIntersected(r1: ValueInterval, r2: ValueInterval): Boolean = (r1, r2) match {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala:79: match may not be exhaustive.
      [error] It would fail on the following inputs: (NumericValueInterval(_, _), _), (_, NumericValueInterval(_, _)), (_, _)
      [error] [warn]     (r1, r2) match {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala:67: match may not be exhaustive.
      [error] It would fail on the following inputs: (ArrayType(_, _), _), (_, ArrayData()), (_, _)
      [error] [warn]     (endpointsExpression.dataType, endpointsExpression.eval()) match {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala:470: match may not be exhaustive.
      [error] It would fail on the following inputs: NewFunctionSpec(_, None, Some(_)), NewFunctionSpec(_, Some(_), None)
      [error] [warn]     newFunction match {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:94: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely always compare unequal
      [error] [warn]         if (eval.isNull != "true") {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:126: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
      [error] [warn]              if (eval.isNull == "true") {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:133: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
      [error] [warn]             if (eval.isNull == "true") {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala:709: match may not be exhaustive.
      [error] It would fail on the following input: Schema((x: org.apache.spark.sql.types.DataType forSome x not in org.apache.spark.sql.types.StructType), _)
      [error] [warn]   def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match {
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala:90: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
      [error] [warn]       if (inputs.map(_.isNull).forall(_ == "false")) {
      [error] [warn] 
      
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala:410: discarding unmoored doc comment
      [error] [warn]     /**
      [error] [warn] 
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala:441: discarding unmoored doc comment
      [error] [warn]     /**
      [error] [warn] 
      ...
      [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala:440: discarding unmoored doc comment
      [error] [warn]     /**
      [error] [warn] 
      

      Attachments

        Activity

          People

            kiszk Kazuaki Ishizaki
            kiszk Kazuaki Ishizaki
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: