[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988787565


##
connector/connect/src/main/protobuf/spark/connect/expressions.proto:
##
@@ -159,4 +159,17 @@ message Expression {
   // UnresolvedStar is used to expand all the fields of a relation or struct.
   message UnresolvedStar {
   }
+
+  // An resolved attribute that can specify a reference (e.g. column) without 
needing a resolution
+  // by the analyzer.
+  message Attribute {

Review Comment:
   The only reason to have this is to support `LocalRelation`. It's unclear how 
the client can send local data (e.g. java array, pandas dataframe) to the 
server, which is the opposite of `df.collect`, so this is most likely a testing 
util in the near 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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988772280


##
connector/connect/src/main/scala/org/apache/spark/sql/catalyst/connect/connect.scala:
##
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto

Review Comment:
   This is package `org.apache.spark.sql.catalyst` under the `connect` module, 
so there is no circular reference. But it seems no reason to put tests under 
the `catalyst` package. Maybe we can just put test in package 
`org.apache.spark.connect.proto` and then we don't need this import.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988626062


##
connector/connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala:
##
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.connect.planner
+
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.types.{DataType, IntegerType}
+
+/**
+ * This suite is based on connect DSL and test that given same dataframe 
operations, whether
+ * connect could construct a proto plan that can be translated back, and after 
analyzed, be the
+ * same as Spark dataframe's generated plan.
+ */
+class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest {
+
+  lazy val connectTestRelation = createLocalRelationProto(Map("id" -> 
IntegerType))
+
+  lazy val sparkTestRelation: LocalRelation = 
LocalRelation(AttributeReference("id", IntegerType)())

Review Comment:
   we can do the same for proto local relation to simplify code
   ```
   createLocalRelationProto(Seq($"id".int))
   def createLocalRelationProto(attrs: Seq[AttributeReference]) = {
 for (attr <- attrs) {
   .setName(attr.name)
 }
   }
   ```



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988624766


##
connector/connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala:
##
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.connect.planner
+
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.types.{DataType, IntegerType}
+
+/**
+ * This suite is based on connect DSL and test that given same dataframe 
operations, whether
+ * connect could construct a proto plan that can be translated back, and after 
analyzed, be the
+ * same as Spark dataframe's generated plan.
+ */
+class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest {
+
+  lazy val connectTestRelation = createLocalRelationProto(Map("id" -> 
IntegerType))
+
+  lazy val sparkTestRelation: LocalRelation = 
LocalRelation(AttributeReference("id", IntegerType)())

Review Comment:
   we can use dsl here: `LocalRelation($"id".int)`



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988622964


##
connector/connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##
@@ -111,8 +122,12 @@ class SparkConnectPlanner(plan: proto.Relation, session: 
SparkSession) {
 }
   }
 
-  private def transformUnresolvedExpression(exp: proto.Expression): 
UnresolvedAttribute = {
-UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  private def transformUnresolvedExpression(exp: proto.Expression): Expression 
= {
+if (exp.getUnresolvedAttribute.getPartsCount == 1) {
+  Literal.create(exp.getUnresolvedAttribute.getParts(0), StringType)

Review Comment:
   ideally the catalyst dsl should do `relation.select($"a")` to indicate this 
selects a column.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988619447


##
connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala:
##
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.connect.planner
+
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.types.{DataType, IntegerType}
+
+/**
+ * This suite is based on connect DSL and test that given same dataframe 
operations, whether
+ * connect could construct a proto plan that can be translated back, and after 
analyzed, be the
+ * same as Spark dataframe's generated plan.
+ */
+class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest {
+
+  lazy val connectTestRelation = createLocalRelationProto(Map("id" -> 
IntegerType))
+
+  lazy val sparkTestRelation: LocalRelation = 
LocalRelation(AttributeReference("id", IntegerType)())
+
+  test("Basic select") {
+val connectPlan = {
+  // TODO: Scala only allows one implicit per scope so we keep proto 
implicit imports in
+  // this scope. Need to find a better way to make two implicits work in 
the smae scope.
+  import org.apache.spark.sql.catalyst.connect.expressions._
+  import org.apache.spark.sql.catalyst.connect.plans._
+  transform(connectTestRelation.select("id".protoAttr))
+}
+val sparkPlan = sparkTestRelation.select("id")
+comparePlans(connectPlan.analyze, sparkPlan.analyze, false)
+  }
+
+  private def createLocalRelationProto(colNameTypeMap: Map[String, DataType]): 
proto.Relation = {

Review Comment:
   it's weird to pass a `Map` here, as the elements order is not deterministic. 
We should use `Seq[(String, DataType)]` or simply `Seq[AttributeReference]`



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-10-06 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r988618331


##
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##
@@ -109,8 +120,12 @@ class SparkConnectPlanner(plan: proto.Relation, session: 
SparkSession) {
 }
   }
 
-  private def transformUnresolvedExpression(exp: proto.Expression): 
UnresolvedAttribute = {
-UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  private def transformUnresolvedExpression(exp: proto.Expression): Expression 
= {
+if (exp.getUnresolvedAttribute.getPartsCount == 1) {
+  Literal.create(exp.getUnresolvedAttribute.getParts(0), StringType)

Review Comment:
   We can clean up this part later. This special case for string literal is 
very weird.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-28 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r982067988


##
connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala:
##
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.connect.planner
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.connect.plans.DslLogicalPlan
+import org.apache.spark.sql.catalyst.connect.plans.DslString
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * This suite is based on connect DSL and test that given same dataframe 
operations, whether
+ * connect could construct a proto plan that can be translated back, and after 
analyzed, be the
+ * same as Spark dataframe's generated plan.
+ */
+class SparkConnectProtoSuite extends SparkFunSuite
+  with SharedSparkSession with SparkConnectPlanTest with PlanTest with 
BeforeAndAfter {
+
+  lazy val connectTestRelation =
+proto.Relation.newBuilder()
+  .setRead(
+proto.Read.newBuilder()
+
.setNamedTable(proto.Read.NamedTable.newBuilder().addParts("student").build())
+.build())
+  .build()
+
+  lazy val sparkTestRelation = spark.table("student")
+
+  before {
+setupTestData()
+  }
+
+  test("Basic select") {
+val connectPlan = analyze(connectTestRelation.select("id".protoAttr))
+val sparkPlan = sparkTestRelation.select("id").queryExecution.analyzed

Review Comment:
   For example, `AnalysisSuite` is a pure logical plan test with dsl.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-28 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r982067325


##
connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala:
##
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.connect.planner
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.connect.plans.DslLogicalPlan
+import org.apache.spark.sql.catalyst.connect.plans.DslString
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * This suite is based on connect DSL and test that given same dataframe 
operations, whether
+ * connect could construct a proto plan that can be translated back, and after 
analyzed, be the
+ * same as Spark dataframe's generated plan.
+ */
+class SparkConnectProtoSuite extends SparkFunSuite
+  with SharedSparkSession with SparkConnectPlanTest with PlanTest with 
BeforeAndAfter {
+
+  lazy val connectTestRelation =
+proto.Relation.newBuilder()
+  .setRead(
+proto.Read.newBuilder()
+
.setNamedTable(proto.Read.NamedTable.newBuilder().addParts("student").build())
+.build())
+  .build()
+
+  lazy val sparkTestRelation = spark.table("student")
+
+  before {
+setupTestData()
+  }
+
+  test("Basic select") {
+val connectPlan = analyze(connectTestRelation.select("id".protoAttr))
+val sparkPlan = sparkTestRelation.select("id").queryExecution.analyzed

Review Comment:
   do we really need DataFrame here? We can use pure logical plans:
   ```
   val testRelation = LocalRelation(...)
   ...
   val connectPlan = transform(...).analyze
   val sparkPlan = testRelation.select... // using the catalyst dsl, not 
DataFrame APIs
   ```



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-28 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r982063973


##
connect/src/main/scala/org/apache/spark/sql/catalyst/connect/connect.scala:
##
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+
+/**
+ * A collection of implicit conversions that create a DSL for constructing 
connect protos.
+ */
+package object connect {
+
+  object plans { // scalastyle:ignore
+implicit class DslString(val s: String) {
+  def protoAttr: proto.Expression =
+proto.Expression.newBuilder()
+  .setUnresolvedAttribute(
+proto.Expression.UnresolvedAttribute.newBuilder()
+  .addParts(s)

Review Comment:
   shall we call `CatalystSqlParser.parseMultipartIdentifier` to support cases 
like `"a.b".protoAttr`?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-28 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r982062517


##
connect/src/main/scala/org/apache/spark/sql/catalyst/connect/connect.scala:
##
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+
+/**
+ * A collection of implicit conversions that create a DSL for constructing 
connect protos.
+ */
+package object connect {
+
+  object plans { // scalastyle:ignore

Review Comment:
   hmmm, do we need this `object plans`? If we want to follow catalyst dsl 
completely, we should have both `object plans` and `object expressions`, so 
that callers can import plan and expression dsl individually.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-27 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r981913805


##
connect/src/main/scala/org/apache/spark/sql/catalyst/connect/connect.scala:
##
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+
+/**
+ * A collection of implicit conversions that create a DSL for constructing 
connect protos.
+ *
+ * {{{
+ *   scala> import org.apache.spark.sql.connect.plans.DslLogicalPlan
+ *
+ *   // Standard way to construct connect proto
+ *   scala> import org.apache.spark.connect.proto
+ *   scala> :paste
+ *   // Entering paste mode (ctrl-D to finish)
+ *   val connectTestRelation =
+ *proto.Relation.newBuilder()
+ * .setRead(
+ *   proto.Read.newBuilder()
+ *   
.setNamedTable(proto.Read.NamedTable.newBuilder().addParts("student").build())
+ *   .build())
+ * .build()
+ *  // Exiting paste mode, now interpreting.
+ *connectTestRelation: org.apache.spark.connect.proto.Relation =
+ * read {
+ *  named_table {
+ *parts: "student"
+ *  }
+ *}
+ *
+ *   // Now we can apply select on the proto relation above
+ *   scala> import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+ *   scala> connectTestRelation.select(UnresolvedAttribute(Seq("id")))
+ *   res14: org.apache.spark.connect.proto.Relation =
+ *project {
+ *  input {
+ *read {
+ *  named_table {
+ *parts: "student"
+ *  }
+ *}
+ *  }
+ *  expressions {
+ *unresolved_attribute {
+ *  parts: "id"
+ *}
+ *  }
+ *}
+ *
+ * }}}
+ *
+ */
+package object connect {
+
+  object plans { // scalastyle:ignore
+implicit class DslLogicalPlan(val logicalPlan: proto.Relation) {
+  def select(exprs: Expression*): proto.Relation = {
+val namedExpressions = exprs.map {
+  case e: NamedExpression => e
+  case e => UnresolvedAlias(e)
+}
+
+proto.Relation.newBuilder().setProject(
+  proto.Project.newBuilder()
+.setInput(logicalPlan)
+.addExpressions(
+  proto.Expression.newBuilder()
+.setUnresolvedAttribute(

Review Comment:
   Sorry I'm still a bit confused about this. How are we going to test more 
expressions in the future? Are we going to put a catalyst `Expression` to proto 
expression converter here?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-27 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r981913250


##
repl/pom.xml:
##
@@ -58,6 +58,11 @@
   spark-sql_${scala.binary.version}
   ${project.version}
 
+
+  org.apache.spark
+  spark-connect_${scala.binary.version}

Review Comment:
   We don't need to support this dsl in Spar repl. We can remove this and also 
move the dsl to `connect/src/test`



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980616391


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+
+
+/**
+ * A collection of implicit conversions that create a DSL for constructing 
connect protos.
+ *
+ * {{{
+ *   scala> import org.apache.spark.sql.connect.plans.DslLogicalPlan
+ *
+ *   // Standard way to construct connect proto
+ *   scala> import org.apache.spark.connect.proto
+ *   scala> :paste
+ *   // Entering paste mode (ctrl-D to finish)
+ *   val connectTestRelation =
+ *proto.Relation.newBuilder()
+ * .setRead(
+ *   proto.Read.newBuilder()
+ *   
.setNamedTable(proto.Read.NamedTable.newBuilder().addParts("student").build())
+ *   .build())
+ * .build()
+ *  // Exiting paste mode, now interpreting.
+ *connectTestRelation: org.apache.spark.connect.proto.Relation =
+ * read {
+ *  named_table {
+ *parts: "student"
+ *  }
+ *}
+ *
+ *   // Now we can apply select on the proto relation above
+ *   scala> import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+ *   scala> connectTestRelation.select(UnresolvedAttribute(Seq("id")))
+ *   res14: org.apache.spark.connect.proto.Relation =
+ *project {
+ *  input {
+ *read {
+ *  named_table {
+ *parts: "student"
+ *  }
+ *}
+ *  }
+ *  expressions {
+ *unresolved_attribute {
+ *  parts: "id"
+ *}
+ *  }
+ *}
+ *
+ * }}}
+ *
+ */
+package object connect {
+
+  object plans { // scalastyle:ignore
+implicit class DslLogicalPlan(val logicalPlan: proto.Relation) {

Review Comment:
   Does `proto.Relation` mean logical plan in the proto world?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980615672


##
connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala:
##
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.connect.planner
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.connect.plans.DslLogicalPlan
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * This suite is based on connect DSL and test that given same dataframe 
operations, whether
+ * connect could construct a proto plan that can be translated back, and after 
analyzed, be the
+ * same as Spark dataframe's generated plan.
+ */
+class SparkConnectProtoSuite extends SparkFunSuite
+  with SharedSparkSession with SparkConnectPlanTest with PlanTest with 
BeforeAndAfter {
+
+  lazy val connectTestRelation =
+proto.Relation.newBuilder()
+  .setRead(
+proto.Read.newBuilder()
+
.setNamedTable(proto.Read.NamedTable.newBuilder().addParts("student").build())
+.build())
+  .build()
+
+  lazy val sparkTestRelation = spark.table("student")
+
+  before {
+setupTestData()
+  }
+
+  test("Basic select") {
+val connectPlan = 
analyze(connectTestRelation.select(UnresolvedAttribute(Seq("id"
+val sparkPlan = sparkTestRelation.select("id").queryExecution.analyzed
+comparePlans(connectPlan, sparkPlan)
+  }
+
+  private def analyze(plan: proto.Relation): LogicalPlan = {
+spark.sessionState.executePlan(transform(plan)).analyzed
+  }
+
+  protected override def comparePlans(
+plan1: LogicalPlan,

Review Comment:
   nit: 4 spaces indentation



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980615123


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql

Review Comment:
   BTW, I don't think we have to make it available in spark-shell. The catalyst 
dsl was added before we have DataFrame API. At that time, it was the only way 
for end-users to build query plan with a dataframe-like API. We don't have such 
a need for spark connect dsl.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980615483


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+
+
+/**
+ * A collection of implicit conversions that create a DSL for constructing 
connect protos.
+ *
+ * {{{
+ *   scala> import org.apache.spark.sql.connect.plans.DslLogicalPlan
+ *
+ *   // Standard way to construct connect proto
+ *   scala> import org.apache.spark.connect.proto
+ *   scala> :paste
+ *   // Entering paste mode (ctrl-D to finish)
+ *   val connectTestRelation =
+ *proto.Relation.newBuilder()
+ * .setRead(
+ *   proto.Read.newBuilder()
+ *   
.setNamedTable(proto.Read.NamedTable.newBuilder().addParts("student").build())
+ *   .build())
+ * .build()
+ *  // Exiting paste mode, now interpreting.
+ *connectTestRelation: org.apache.spark.connect.proto.Relation =
+ * read {
+ *  named_table {
+ *parts: "student"
+ *  }
+ *}
+ *
+ *   // Now we can apply select on the proto relation above
+ *   scala> import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+ *   scala> connectTestRelation.select(UnresolvedAttribute(Seq("id")))
+ *   res14: org.apache.spark.connect.proto.Relation =
+ *project {
+ *  input {
+ *read {
+ *  named_table {
+ *parts: "student"
+ *  }
+ *}
+ *  }
+ *  expressions {
+ *unresolved_attribute {
+ *  parts: "id"
+ *}
+ *  }
+ *}
+ *
+ * }}}
+ *
+ */
+package object connect {
+
+  object plans { // scalastyle:ignore
+implicit class DslLogicalPlan(val logicalPlan: proto.Relation) {
+  def select(exprs: Expression*): proto.Relation = {

Review Comment:
   should this be `proto.Expression`? Will we ever add `proto.Expression`?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980615123


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql

Review Comment:
   BTW, I don't think we have to make it available in spark-shell. The catalyst 
dsl was added before we have DataFrame API. At that time, it was the only way 
for end-users to build query plan with a dataframe-like API.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980614606


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql

Review Comment:
   yup



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980610466


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql

Review Comment:
   The dsl should not be a public API. Can we move it to 
`org.apache.spark.sql.catalyst` which is a private package?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] cloud-fan commented on a diff in pull request #37994: [SPARK-40454][CONNECT] Initial DSL framework for protobuf testing

2022-09-26 Thread GitBox


cloud-fan commented on code in PR #37994:
URL: https://github.com/apache/spark/pull/37994#discussion_r980206002


##
connect/src/main/scala/org/apache/spark/sql/connect/package.scala:
##
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import org.apache.spark.connect.proto
+

Review Comment:
   yea, `def select` should take `Expression` instead of `String`



-- 
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-unsubscr...@spark.apache.org

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


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