You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "dtenedor (via GitHub)" <gi...@apache.org> on 2023/07/06 16:52:32 UTC

[GitHub] [spark] dtenedor commented on a diff in pull request #41750: [SPARK-44200][SQL] Support TABLE argument parser rule for TableValuedFunction

dtenedor commented on code in PR #41750:
URL: https://github.com/apache/spark/pull/41750#discussion_r1254694064


##########
python/pyspark/sql/tests/test_udtf.py:
##########
@@ -397,6 +397,198 @@ def test_udtf(a: int):
         with self.assertRaisesRegex(TypeError, err_msg):
             udtf(test_udtf, returnType="a: int")
 
+    def test_udtf_with_table_argument_query(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+        self.assertEqual(
+            self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT id FROM range(0, 8)))").collect(),
+            [Row(a=6), Row(a=7)],
+        )
+
+    def test_udtf_with_int_and_table_argument_query(self):
+        class TestUDTF:
+            def eval(self, i: int, row: Row):
+                if row["id"] > i:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+        self.assertEqual(
+            self.spark.sql(
+                "SELECT * FROM test_udtf(5, TABLE (SELECT id FROM range(0, 8)))"
+            ).collect(),
+            [Row(a=6), Row(a=7)],
+        )
+
+    def test_udtf_with_table_argument_identifier(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+
+        with self.tempView("v"):
+            self.spark.sql("CREATE OR REPLACE TEMPORARY VIEW v as SELECT id FROM range(0, 8)")
+            self.assertEqual(
+                self.spark.sql("SELECT * FROM test_udtf(TABLE v)").collect(),
+                [Row(a=6), Row(a=7)],
+            )
+
+    def test_udtf_with_int_and_table_argument_identifier(self):
+        class TestUDTF:
+            def eval(self, i: int, row: Row):
+                if row["id"] > i:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+
+        with self.tempView("v"):
+            self.spark.sql("CREATE OR REPLACE TEMPORARY VIEW v as SELECT id FROM range(0, 8)")
+            self.assertEqual(
+                self.spark.sql("SELECT * FROM test_udtf(5, TABLE v)").collect(),
+                [Row(a=6), Row(a=7)],
+            )
+
+    def test_udtf_with_table_argument_unknown_identifier(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+
+        with self.assertRaisesRegex(AnalysisException, "TABLE_OR_VIEW_NOT_FOUND"):
+            self.spark.sql("SELECT * FROM test_udtf(TABLE v)").collect()
+
+    def test_udtf_with_table_argument_malformed_query(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+
+        with self.assertRaisesRegex(AnalysisException, "TABLE_OR_VIEW_NOT_FOUND"):
+            self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT * FROM v))").collect()
+
+    def test_udtf_with_table_argument_cte_inside(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+        self.assertEqual(
+            self.spark.sql(
+                """
+                SELECT * FROM test_udtf(TABLE (
+                  WITH t AS (
+                    SELECT id FROM range(0, 8)
+                  )
+                  SELECT * FROM t
+                ))
+                """
+            ).collect(),
+            [Row(a=6), Row(a=7)],
+        )
+
+    def test_udtf_with_table_argument_cte_outside(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+        self.assertEqual(
+            self.spark.sql(
+                """
+                WITH t AS (
+                  SELECT id FROM range(0, 8)
+                )
+                SELECT * FROM test_udtf(TABLE (SELECT id FROM t))
+                """
+            ).collect(),
+            [Row(a=6), Row(a=7)],
+        )
+
+        self.assertEqual(
+            self.spark.sql(
+                """
+                WITH t AS (
+                  SELECT id FROM range(0, 8)
+                )
+                SELECT * FROM test_udtf(TABLE t)
+                """
+            ).collect(),
+            [Row(a=6), Row(a=7)],
+        )
+
+    # TODO(SPARK-44233): Fix the subquery resolution.
+    @unittest.skip("Fails to resolve the subquery.")
+    def test_udtf_with_table_argument_lateral_join(self):
+        class TestUDTF:
+            def eval(self, row: Row):
+                if row["id"] > 5:
+                    yield row["id"],
+
+        func = udtf(TestUDTF, returnType="a: int")
+        self.spark.udtf.register("test_udtf", func)
+        self.assertEqual(
+            self.spark.sql(
+                """
+                SELECT * FROM
+                  range(0, 8) AS t,
+                  LATERAL test_udtf(TABLE t)
+                """
+            ).collect(),
+            [Row(a=6), Row(a=7)],
+        )
+
+    def test_udtf_with_table_argument_multiple(self):
+        class TestUDTF:
+            def eval(self, a: Row, b: Row):
+                yield a[0], b[0]
+
+        func = udtf(TestUDTF, returnType="a: int, b: int")
+        self.spark.udtf.register("test_udtf", func)
+
+        query = """
+          SELECT * FROM test_udtf(
+            TABLE (SELECT id FROM range(0, 2)),
+            TABLE (SELECT id FROM range(0, 3)))
+        """
+
+        with self.sql_conf({"spark.sql.tvf.allowMultipleTableArguments.enabled": False}):

Review Comment:
   The idea is that we want to support the TABLE keyword for providing a relation argument to a UDTF. If we have exactly one such argument, it behaves the same way as doing a LATERAL JOIN from that relation to the UDTF and then projecting out only the result columns from the UDTF. However, trying this with two TABLE arguments, @ueshin found that the result was equivalent to doing a full cross join between the two relations and then feeding that into the UDTF instead.
   
   We figured that this latter behavior is rather surprising, so we opted to block it with a config (off by default) to return an error in this case. We can then revisit the semantics we'd like to have in the case of multiple TABLE arguments. For example, we could make some Python UDTF API support for this in a way that makes sense. But the intention in that case would almost certainly not to perform a full cross join between the input TABLEs, but instead to plan some operator with multiple child operators that could consume from each of the inputs at will. We can brainstorm about this in the future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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