XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.3.0
    • 2.3.0
    • SQL
    • None

    Description

      Hive pmod(3.13, 0):

      hive> select pmod(3.13, 0);
      OK
      NULL
      Time taken: 2.514 seconds, Fetched: 1 row(s)
      hive> 
      

      Spark mod(3.13, 0):

      spark-sql> select mod(3.13, 0);
      NULL
      spark-sql> 
      

      But the Spark pmod(3.13, 0):

      spark-sql> select pmod(3.13, 0);
      17/06/25 09:35:58 ERROR SparkSQLDriver: Failed in [select pmod(3.13, 0)]
      java.lang.NullPointerException
      	at org.apache.spark.sql.catalyst.expressions.Pmod.pmod(arithmetic.scala:504)
      	at org.apache.spark.sql.catalyst.expressions.Pmod.nullSafeEval(arithmetic.scala:432)
      	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:419)
      	at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:323)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$transformExpressionsDown$1.apply(QueryPlan.scala:82)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$transformExpressionsDown$1.apply(QueryPlan.scala:82)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:103)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:113)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1$1.apply(QueryPlan.scala:117)
      	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      	at scala.collection.immutable.List.foreach(List.scala:381)
      	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
      	at scala.collection.immutable.List.map(List.scala:285)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:117)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$1.apply(QueryPlan.scala:122)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:122)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:82)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:45)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:44)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:44)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:43)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82)
      	at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
      	at scala.collection.immutable.List.foldLeft(List.scala:84)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:82)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74)
      	at scala.collection.immutable.List.foreach(List.scala:381)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74)
      	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:78)
      	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:78)
      	at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:216)
      	at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:216)
      	at org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:112)
      	at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:216)
      	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:340)
      	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:248)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
      	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 org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:755)
      	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:119)
      	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      java.lang.NullPointerException
      	at org.apache.spark.sql.catalyst.expressions.Pmod.pmod(arithmetic.scala:504)
      	at org.apache.spark.sql.catalyst.expressions.Pmod.nullSafeEval(arithmetic.scala:432)
      	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:419)
      	at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:323)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$transformExpressionsDown$1.apply(QueryPlan.scala:82)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$transformExpressionsDown$1.apply(QueryPlan.scala:82)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:103)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:113)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1$1.apply(QueryPlan.scala:117)
      	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      	at scala.collection.immutable.List.foreach(List.scala:381)
      	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
      	at scala.collection.immutable.List.map(List.scala:285)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$recursiveTransform$1(QueryPlan.scala:117)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$1.apply(QueryPlan.scala:122)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:122)
      	at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:82)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:45)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:44)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
      	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
      	at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:44)
      	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:43)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82)
      	at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
      	at scala.collection.immutable.List.foldLeft(List.scala:84)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:82)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74)
      	at scala.collection.immutable.List.foreach(List.scala:381)
      	at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74)
      	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:78)
      	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:78)
      	at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:216)
      	at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:216)
      	at org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:112)
      	at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:216)
      	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:340)
      	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:248)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
      	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 org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:755)
      	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:119)
      	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      
      spark-sql> 
      

      Attachments

        Issue Links

          Activity

            People

              yumwang Yuming Wang
              yumwang Yuming Wang
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: