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

Inconsistency between Scala and Python/Panda udfs when groupby with udf() is used

    XMLWordPrintableJSON

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 3.0.0
    • Fix Version/s: 3.0.0
    • Component/s: PySpark, SQL
    • Labels:
      None

      Description

      Python:

      from pyspark.sql.functions import pandas_udf, PandasUDFType
      
      @pandas_udf("int", PandasUDFType.SCALAR)
      def noop(x):
       return x
      
      spark.udf.register("udf", noop)
      
      sql("""
       CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
       (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null)
       AS testData(a, b)""")
      
      sql("""SELECT udf(a + 1), udf(COUNT(b)) FROM testData GROUP BY udf(a + 1)""").show()
      
      : org.apache.spark.sql.AnalysisException: expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
      Aggregate [udf((a#0 + 1))], [udf((a#0 + 1)) AS udf((a + 1))#10, udf(count(b#1)) AS udf(count(b))#12]
      +- SubqueryAlias `testdata`
       +- Project [a#0, b#1]
       +- SubqueryAlias `testData`
       +- LocalRelation [a#0, b#1]
      

      Scala:

      spark.udf.register("udf", (input: Int) => input)
      
      sql("""
       CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
       (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null)
       AS testData(a, b)""")
      
      sql("""SELECT udf(a + 1), udf(COUNT(b)) FROM testData GROUP BY udf(a + 1)""").show()
      
      +------------+-------------+
      |udf((a + 1))|udf(count(b))|
      +------------+-------------+
      |        null|            1|
      |           3|            2|
      |           4|            2|
      |           2|            2|
      +------------+-------------+
      

        Attachments

          Issue Links

            Activity

              People

              • Assignee:
                viirya Liang-Chi Hsieh
                Reporter:
                skonto Stavros Kontopoulos
              • Votes:
                0 Vote for this issue
                Watchers:
                4 Start watching this issue

                Dates

                • Created:
                  Updated:
                  Resolved: