You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/10/13 09:09:26 UTC

[GitHub] [spark] zhengruifeng commented on a diff in pull request #38218: [SPARK-40587][CONNECT][FOLLOW] Make sure python client support select *

zhengruifeng commented on code in PR #38218:
URL: https://github.com/apache/spark/pull/38218#discussion_r994370396


##########
python/pyspark/sql/connect/plan.py:
##########
@@ -143,23 +143,29 @@ def __init__(self, child: Optional["LogicalPlan"], *columns: "ExpressionOrString
         self._verify_expressions()
 
     def _verify_expressions(self) -> None:
-        """Ensures that all input arguments are instances of Expression."""
+        """Ensures that all input arguments are instances of Expression or String."""
         for c in self._raw_columns:
-            if not isinstance(c, Expression):
-                raise InputValidationError(f"Only Expressions can be used for projections: '{c}'.")
+            if not isinstance(c, Expression) and not isinstance(c, str):
+                raise InputValidationError(
+                    f"Only Expressions or String can be used for projections: '{c}'."
+                )
 
     def withAlias(self, alias: str) -> LogicalPlan:
         self.alias = alias
         return self
 
     def plan(self, session: Optional["RemoteSparkSession"]) -> proto.Relation:
         assert self._child is not None
-        proj_exprs = [
-            c.to_plan(session)
-            if isinstance(c, Expression)
-            else self.unresolved_attr(*(c.split(".")))
-            for c in self._raw_columns
-        ]
+        proj_exprs = []
+        for c in self._raw_columns:
+            if isinstance(c, Expression):
+                proj_exprs.append(c.to_plan(session))
+            elif c == "*":
+                exp = proto.Expression()
+                exp.unresolved_star.SetInParent()
+                proj_exprs.append(exp)
+            else:
+                proj_exprs.append(self.unresolved_attr(*(c.split("."))))

Review Comment:
   what about adding a single UT covering all the 3 cases?



##########
python/pyspark/sql/connect/plan.py:
##########
@@ -143,23 +143,29 @@ def __init__(self, child: Optional["LogicalPlan"], *columns: "ExpressionOrString
         self._verify_expressions()
 
     def _verify_expressions(self) -> None:
-        """Ensures that all input arguments are instances of Expression."""
+        """Ensures that all input arguments are instances of Expression or String."""
         for c in self._raw_columns:
-            if not isinstance(c, Expression):
-                raise InputValidationError(f"Only Expressions can be used for projections: '{c}'.")
+            if not isinstance(c, Expression) and not isinstance(c, str):

Review Comment:
   ```suggestion
               if not isinstance(c, (Expression, str)):
   ```



##########
python/pyspark/sql/tests/test_connect_basic.py:
##########
@@ -79,6 +79,11 @@ def test_simple_explain_string(self):
         result = df.explain()
         self.assertGreater(len(result), 0)
 
+    def test_select_start(self):
+        df = self.connect.read.table(self.tbl_name).select("*")
+        data = df.toPandas()
+        self.assertEqual(len(data.index), 100)

Review Comment:
   so it can not support selecting `struct.*` now?



-- 
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