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/08/29 16:34:31 UTC

[GitHub] [spark] grundprinzip opened a new pull request, #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

grundprinzip opened a new pull request, #37710:
URL: https://github.com/apache/spark/pull/37710

   ### What changes were proposed in this pull request?
   This is a draft change of the current state of the Spark Connect prototype implemented as a driver plugin to separate the classpaths and shaded dependent libraries.
   
   ### Why are the changes needed?
   
   https://issues.apache.org/jira/browse/SPARK-39375
   
   ### Does this PR introduce _any_ user-facing change?
   Experimental API for Spark Connect
   
   ### How was this patch tested?
   
   Will be added.
   


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


[GitHub] [spark] grundprinzip commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1257437750

   Ack, I will regenerate the protos and update.


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


[GitHub] [spark] pan3793 commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975553286


##########
dev/deps/spark-deps-hadoop-3-hive-2.3:
##########
@@ -60,10 +62,20 @@ datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar
 datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar
 derby/10.14.2.0//derby-10.14.2.0.jar
 dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar
+error_prone_annotations/2.10.0//error_prone_annotations-2.10.0.jar
+failureaccess/1.0.1//failureaccess-1.0.1.jar
 flatbuffers-java/1.12.0//flatbuffers-java-1.12.0.jar
 gcs-connector/hadoop3-2.2.7/shaded/gcs-connector-hadoop3-2.2.7-shaded.jar
 generex/1.0.2//generex-1.0.2.jar
 gmetric4j/1.0.10//gmetric4j-1.0.10.jar
+grpc-api/1.47.0//grpc-api-1.47.0.jar

Review Comment:
   As you are intending to shade grpc into the connect jar, why jars still are there?



##########
dev/infra/Dockerfile:
##########
@@ -65,3 +65,6 @@ RUN Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='ht
 
 # See more in SPARK-39735
 ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library"
+
+# Add Python Deps for Spark Connect.
+RUN python3.9 -m pip install grpcio protobuf

Review Comment:
   nit: new line



##########
project/SparkBuild.scala:
##########
@@ -474,9 +488,11 @@ object SparkBuild extends PomBuild {
 
     sparkSql := {
       (Compile / runMain).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value
-    }
+    },
+
   ))(assembly)
 
+

Review Comment:
   unnecessary change



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>

Review Comment:
   should be `${guava.version}`?



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>

Review Comment:
   it's unnecessary as it is already configured in the parent pom



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <argLine>-ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true</argLine>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala-${scala.binary.version}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-scala-test-sources</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/gen-java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>0.6.1</version>
+                <configuration>
+                    <protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+                    <pluginId>grpc-java</pluginId>
+                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:${io.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+                    <protoSourceRoot>src/main/protobuf</protoSourceRoot>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>compile</goal>
+                            <goal>compile-custom</goal>
+                            <goal>test-compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Shade all GRPC / Guava / Protobuf depencies of this build -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <configuration>
+                    <shadedArtifactAttached>false</shadedArtifactAttached>
+                    <artifactSet>
+                        <includes>
+                            <include>com.google.guava:*</include>
+                            <include>io.grpc:*:</include>
+                            <include>com.google.protobuf:*</include>
+                        </includes>
+                    </artifactSet>
+                    <relocations>
+                        <relocation>
+                            <pattern>com.google.common</pattern>

Review Comment:
   it leaks `com.google.thirdparty` classes



##########
connect/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+import "spark/connect/types.proto";
+
+package spark.connect;
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+// A [[Command]] is an operation that is executed by the server that does not directly consume or
+// produce a relational result.
+message Command {
+  oneof command_type {
+    CreateScalarFunction create_function = 1;
+  }
+}
+
+// Simple message that is used to create a scalar function based on the provided function body.
+//
+// This message is used to register for example a Python UDF in the session catalog by providing
+// the serialized method body.
+message CreateScalarFunction {
+  // Fully qualified name of the function including the catalog / schema names.
+  repeated string parts = 1;
+  FunctionLanguage language = 2;
+  bool temporary = 3;
+  repeated Type argument_types = 4;
+  Type return_type = 5;
+
+  // How the function body is defined:
+  oneof function_definition {
+    // As a raw string serialized:
+    bytes serialized_function = 6;
+    // As a code literal
+    string literal_string = 7;
+  }
+
+  enum FunctionLanguage {
+    FUNCTION_LANGUAGE_UNSPECIFIED = 0;
+    FUNCTION_LANGUAGE_SQL = 1;
+    FUNCTION_LANGUAGE_PYTHON = 2;
+    FUNCTION_LANGUAGE_SCALA = 3;
+  }
+}

Review Comment:
   nit: new line



##########
python/mypy.ini:
##########
@@ -110,6 +120,7 @@ strict_optional = False
 [mypy-pyspark.cloudpickle.*]
 ignore_errors = True
 
+

Review Comment:
   unnecessary change



##########
project/SparkBuild.scala:
##########
@@ -357,7 +366,10 @@ object SparkBuild extends PomBuild {
 
     // To prevent intermittent compilation failures, see also SPARK-33297
     // Apparently we can remove this when we use JDK 11.
-    Test / classLoaderLayeringStrategy := ClassLoaderLayeringStrategy.Flat
+    Test / classLoaderLayeringStrategy := ClassLoaderLayeringStrategy.Flat,
+
+    // BUG fuck me

Review Comment:
   remove the aggressive words, please



##########
project/SparkBuild.scala:
##########
@@ -474,9 +488,11 @@ object SparkBuild extends PomBuild {
 
     sparkSql := {
       (Compile / runMain).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value
-    }
+    },
+

Review Comment:
   unnecessary change



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>

Review Comment:
   please indent by 2 spaces, as other modules do.



##########
python/mypy.ini:
##########
@@ -138,3 +149,10 @@ ignore_missing_imports = True
 
 [mypy-tabulate.*]
 ignore_missing_imports = True
+
+[mypy-google.protobuf.*]
+ignore_missing_imports = True
+
+; Ignore errors for proto generated code
+[mypy-pyspark.sql.connect.proto.*, pyspark.sql.connect.proto]
+ignore_errors = True

Review Comment:
   nit: new line



##########
connect/src/main/buf.work.yaml:
##########
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+version: v1
+directories:
+  - protobuf

Review Comment:
   nit: new line



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>

Review Comment:
   ditto



##########
connect/src/main/protobuf/google/protobuf/any.proto:
##########
@@ -0,0 +1,155 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option go_package = "github.com/golang/protobuf/ptypes/any";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "AnyProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+
+// `Any` contains an arbitrary serialized protocol buffer message along with a
+// URL that describes the type of the serialized message.
+//
+// Protobuf library provides support to pack/unpack Any values in the form
+// of utility functions or additional generated methods of the Any type.
+//
+// Example 1: Pack and unpack a message in C++.
+//
+//     Foo foo = ...;
+//     Any any;
+//     any.PackFrom(foo);
+//     ...
+//     if (any.UnpackTo(&foo)) {
+//       ...
+//     }
+//
+// Example 2: Pack and unpack a message in Java.
+//
+//     Foo foo = ...;
+//     Any any = Any.pack(foo);
+//     ...
+//     if (any.is(Foo.class)) {
+//       foo = any.unpack(Foo.class);
+//     }
+//
+//  Example 3: Pack and unpack a message in Python.
+//
+//     foo = Foo(...)
+//     any = Any()
+//     any.Pack(foo)
+//     ...
+//     if any.Is(Foo.DESCRIPTOR):
+//       any.Unpack(foo)
+//       ...
+//
+//  Example 4: Pack and unpack a message in Go
+//
+//      foo := &pb.Foo{...}
+//      any, err := ptypes.MarshalAny(foo)
+//      ...
+//      foo := &pb.Foo{}
+//      if err := ptypes.UnmarshalAny(any, foo); err != nil {
+//        ...
+//      }
+//
+// The pack methods provided by protobuf library will by default use
+// 'type.googleapis.com/full.type.name' as the type URL and the unpack
+// methods only use the fully qualified type name after the last '/'
+// in the type URL, for example "foo.bar.com/x/y.z" will yield type
+// name "y.z".
+//
+//
+// JSON
+// ====
+// The JSON representation of an `Any` value uses the regular
+// representation of the deserialized, embedded message, with an
+// additional field `@type` which contains the type URL. Example:
+//
+//     package google.profile;
+//     message Person {
+//       string first_name = 1;
+//       string last_name = 2;
+//     }
+//
+//     {
+//       "@type": "type.googleapis.com/google.profile.Person",
+//       "firstName": <string>,
+//       "lastName": <string>
+//     }
+//
+// If the embedded message type is well-known and has a custom JSON
+// representation, that representation will be embedded adding a field
+// `value` which holds the custom JSON in addition to the `@type`
+// field. Example (for message [google.protobuf.Duration][]):
+//
+//     {
+//       "@type": "type.googleapis.com/google.protobuf.Duration",
+//       "value": "1.212s"
+//     }
+//
+message Any {
+  // A URL/resource name that uniquely identifies the type of the serialized
+  // protocol buffer message. This string must contain at least
+  // one "/" character. The last segment of the URL's path must represent
+  // the fully qualified name of the type (as in
+  // `path/google.protobuf.Duration`). The name should be in a canonical form
+  // (e.g., leading "." is not accepted).
+  //
+  // In practice, teams usually precompile into the binary all types that they
+  // expect it to use in the context of Any. However, for URLs which use the
+  // scheme `http`, `https`, or no scheme, one can optionally set up a type
+  // server that maps type URLs to message definitions as follows:
+  //
+  // * If no scheme is provided, `https` is assumed.
+  // * An HTTP GET on the URL must yield a [google.protobuf.Type][]
+  //   value in binary format, or produce an error.
+  // * Applications are allowed to cache lookup results based on the
+  //   URL, or have them precompiled into a binary to avoid any
+  //   lookup. Therefore, binary compatibility needs to be preserved
+  //   on changes to types. (Use versioned type names to manage
+  //   breaking changes.)
+  //
+  // Note: this functionality is not currently available in the official
+  // protobuf release, and it is not used for type URLs beginning with
+  // type.googleapis.com.
+  //
+  // Schemes other than `http`, `https` (or the empty scheme) might be
+  // used with implementation specific semantics.
+  //
+  string type_url = 1;
+
+  // Must be a valid serialized protocol buffer of the above specified type.
+  bytes value = 2;
+}

Review Comment:
   nit: new line



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <argLine>-ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true</argLine>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala-${scala.binary.version}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-scala-test-sources</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/gen-java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>

Review Comment:
   why compile target Java 1.6?



##########
connect/src/main/protobuf/google/protobuf/empty.proto:
##########
@@ -0,0 +1,52 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option go_package = "github.com/golang/protobuf/ptypes/empty";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "EmptyProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+option cc_enable_arenas = true;
+
+// A generic empty message that you can re-use to avoid defining duplicated
+// empty messages in your APIs. A typical example is to use it as the request
+// or the response type of an API method. For instance:
+//
+//     service Foo {
+//       rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty);
+//     }
+//
+// The JSON representation for `Empty` is empty JSON object `{}`.
+message Empty {}

Review Comment:
   nit: new line



##########
connect/src/main/protobuf/spark/connect/base.proto:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/commands.proto";
+import "spark/connect/relations.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+
+// A [[Plan]] is the structure that carries the runtime information for the execution from the
+// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference
+// to the underlying logical plan or it can be of the [[Command]] type that is used to execute
+// commands on the server.
+message Plan {
+  oneof op_type {
+    Relation root = 1;
+    Command command = 2;
+  }
+}
+
+// A request to be executed by the service.
+message Request {
+  // The client_id is set by the client to be able to collate streaming responses from
+  // different queries.
+  string client_id = 1;
+  // User context
+  UserContext user_context = 2;
+  // The logical plan to be executed / analyzed.
+  Plan plan = 3;
+
+  // User Context is used to refer to one particular user session that is executing
+  // queries in the backend.
+  message UserContext {
+    string user_id = 1;
+    string user_name = 2;
+  }
+}
+
+// The response of a query, can be one or more for each request. Responses belonging to the
+// same input query, carry the same `client_id`.
+message Response {
+  string client_id = 1;
+
+  // Result type
+  oneof result_type {
+    ArrowBatch batch = 2;
+    CSVBatch csv_batch = 3;
+  }
+
+  // Metrics for the query execution. Typically, this field is only present in the last
+  // batch of results and then represent the overall state of the query execution.
+  Metrics metrics = 4;
+
+  // Batch results of metrics.
+  message ArrowBatch {
+    int64 row_count = 1;
+    int64 uncompressed_bytes = 2;
+    int64 compressed_bytes = 3;
+    bytes data = 4;
+    bytes schema = 5;
+  }
+
+  message CSVBatch {

Review Comment:
   `CSV` or `Csv`?



##########
project/SparkBuild.scala:
##########
@@ -79,6 +83,11 @@ object BuildCommons {
   val testTempDir = s"$sparkHome/target/tmp"
 
   val javaVersion = settingKey[String]("source and target JVM version for javac and scalac")
+
+  // Google Protobuf version used for generating the protobuf.
+  val protoVersion = "3.21.0"

Review Comment:
   why 3.21.0 used in sbt but 3.21.1 used in maven? is it possible to share the version variable across maven and sbt?



##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.sparkconnect.service
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sparkconnect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.sparkconnect.planner.SparkConnectPlanner
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    // Preconditions.checkState(v.userContext.nonEmpty, "User Context must be present")
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+
+    // TODO empty results (except limit 0) will not yield a schema.
+
+    val data = rows.collect().map(x => x.toSeq.mkString("|")).mkString("\n")
+    val bbb = proto.Response.CSVBatch.newBuilder
+      .setRowCount(-1)
+      .setData(textSchema ++ "\n" ++ data)
+      .build()
+    val response = proto.Response.newBuilder().setClientId(clientId).setCsvBatch(bbb).build()
+
+    // Send all the data
+    responseObserver.onNext(response)
+
+    //    val batches = rows.collectToArrowBatches()

Review Comment:
   please remove the unused code block intead of commenting



##########
connect/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ The main [[Relation]] type. Fundamentally, a relation is a typed container
+ that has exactly one explicit relation type set.
+
+ When adding new relation types, they have to be registered here.
+ */
+message Relation {
+  RelationCommon common = 1;
+  oneof rel_type {
+    Read read = 2;
+    Project project = 3;
+    Filter filter = 4;
+    Join join = 5;
+    Union union = 6;
+    Sort sort = 7;
+    Fetch fetch = 8;
+    Aggregate aggregate = 9;
+    Sql sql = 10;
+
+    Unknown unknown = 999;
+  }
+}
+
+/*
+ * Used for testing purposes only.
+ */
+message Unknown {}
+
+/*
+ Common metadata of all relations.
+ */
+message RelationCommon {
+  string source_info = 1;
+  string alias = 2;
+}
+
+/*
+ Relation that uses a SQL query to generate the output.
+ */
+message Sql {
+  string query = 1;
+}
+
+/*
+ Relation that reads from a file / table or other data source. Does not have additional
+ inputs.
+ */
+message Read {
+  oneof read_type {
+    NamedTable named_table = 1;
+  }
+
+  message NamedTable {
+    repeated string parts = 1;
+  }
+}
+
+/*
+ Projection of a bag of expressions for a given input relation.
+
+ The input relation must be specified.
+ The projected expression can be an arbitrary expression.
+ */
+message Project {
+  Relation input = 1;
+  repeated Expression expressions = 3;
+}
+
+/*
+ Relation that applies a boolean expression `condition` on each row of `input` to produce the output result.
+ */
+message Filter {
+  Relation input = 1;
+  Expression condition = 2;
+}
+
+/*
+ Relation of type [[Join]].
+
+ `left` and `right` must be present.
+ */
+message Join {
+  Relation left = 1;
+  Relation right = 2;
+  Expression on = 3;
+  JoinType how = 4;
+
+  enum JoinType {
+    JOIN_TYPE_UNSPECIFIED = 0;
+    JOIN_TYPE_INNER = 1;
+    JOIN_TYPE_OUTER = 2;
+    JOIN_TYPE_LEFT_OUTER = 3;
+    JOIN_TYPE_RIGHT_OUTER = 4;
+    JOIN_TYPE_ANTI = 5;
+  }
+}
+
+/*
+ Relation of type [[Union]], at least one input must be set.
+ */
+message Union {
+  repeated Relation inputs = 1;
+  UnionType union_type = 2;
+
+  enum UnionType {
+    UNION_TYPE_UNSPECIFIED = 0;
+    UNION_TYPE_DISTINCT = 1;
+    UNION_TYPE_ALL = 2;
+  }
+}
+
+/*
+ Relation of type [[Fetch]] that is used to read `limit` / `offset` rows from the input relation.
+ */
+message Fetch {
+  Relation input = 1;
+  int32 limit = 2;
+  int32 offset = 3;
+}
+
+/*
+ Relation of type [[Aggregate]].
+ */
+message Aggregate {
+  Relation input = 1;
+
+  // Grouping sets are used in rollups
+  repeated GroupingSet grouping_sets = 2;
+
+  // Measures
+  repeated Measure measures = 3;
+
+  message GroupingSet {
+    repeated Expression aggregate_expressions = 1;
+  }
+
+  message Measure {
+    AggregateFunction function = 1;
+    // Conditional filter for SUM(x FILTER WHERE x < 10)
+    Expression filter = 2;
+  }
+
+  message AggregateFunction {
+    string name = 1;
+    repeated Expression arguments = 2;
+  }
+}
+
+/*
+ Relation of type [[Sort]].
+ */
+message Sort {
+  Relation input = 1;
+  repeated SortField sort_fields = 2;
+
+  message SortField {
+    Expression expression = 1;
+    SortDirection direction = 2;
+    SortNulls nulls = 3;
+  }
+
+  enum SortDirection {
+    SORT_DIRECTION_UNSPECIFIED = 0;
+    SORT_DIRECTION_ASCENDING = 1;
+    SORT_DIRECTION_DESCENDING = 2;
+  }
+
+  enum SortNulls {
+    SORT_NULLS_UNSPECIFIED = 0;
+    SORT_NULLS_FIRST = 1;
+    SORT_NULLS_LAST = 2;
+  }
+}

Review Comment:
   nit: new line



##########
connect/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ The main [[Relation]] type. Fundamentally, a relation is a typed container
+ that has exactly one explicit relation type set.
+
+ When adding new relation types, they have to be registered here.
+ */
+message Relation {
+  RelationCommon common = 1;
+  oneof rel_type {
+    Read read = 2;
+    Project project = 3;
+    Filter filter = 4;
+    Join join = 5;
+    Union union = 6;
+    Sort sort = 7;
+    Fetch fetch = 8;
+    Aggregate aggregate = 9;
+    Sql sql = 10;
+
+    Unknown unknown = 999;
+  }
+}
+
+/*
+ * Used for testing purposes only.
+ */
+message Unknown {}
+
+/*
+ Common metadata of all relations.
+ */
+message RelationCommon {
+  string source_info = 1;
+  string alias = 2;
+}
+
+/*
+ Relation that uses a SQL query to generate the output.
+ */
+message Sql {

Review Comment:
   `SQL` or `Sql`?



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


[GitHub] [spark] tgravescs commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
tgravescs commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r974308904


##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.sparkconnect.command
+
+import com.google.common.collect.{Lists, Maps}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.{proto => proto}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {

Review Comment:
   need to add the @Since to all the new apis



##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.sparkconnect.command

Review Comment:
   do we need the component to be "sparkconnect" since its already org.apache.spark... maybe just "connect" would be shorter and just as concise?



##########
connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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

Review Comment:
   this seems to be connect instead of sparkconnect component, was that intentionally different and perhaps you meant them to mean different things?



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978304587


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")

Review Comment:
   Can we have the information of `lit` here in the exception message?



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978310099


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)
+      joinType = plans.Inner,
+      condition = Some(transformExpression(rel.getOn)),
+      hint = logical.JoinHint.NONE)
+  }
+
+  private def transformSort(rel: proto.Sort): LogicalPlan = {
+    assert(rel.getSortFieldsCount > 0, "SortFields must be present.")
+    logical.Sort(
+      child = transformRelation(rel.getInput),
+      global = true,
+      order = rel.getSortFieldsList.asScala.map(transformSortOrderExpression).toSeq)
+  }
+
+  private def transformSortOrderExpression(so: proto.Sort.SortField): expressions.SortOrder = {
+    expressions.SortOrder(
+      child = transformUnresolvedExpression(so.getExpression),
+      direction = so.getDirection match {
+        case proto.Sort.SortDirection.SORT_DIRECTION_DESCENDING => expressions.Descending
+        case _ => expressions.Ascending
+      },
+      nullOrdering = so.getNulls match {
+        case proto.Sort.SortNulls.SORT_NULLS_LAST => expressions.NullsLast
+        case _ => expressions.NullsFirst
+      },
+      sameOrderExpressions = Seq.empty)
+  }
+
+  private def transformAggregate(rel: proto.Aggregate): LogicalPlan = {
+    assert(rel.hasInput)
+    assert(rel.getGroupingSetsCount == 1, "Only one grouping set supported")

Review Comment:
   `supported` -> `is supported`.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978308018


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)

Review Comment:
   Could you add more description please? For example, JIRA title at least?



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978343163


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   This one, I will handle separately. It will be published like Kafka or Avro, and yes it will be covered by a profile. I will do separately.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978362369


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}

Review Comment:
   Just saw https://github.com/apache/spark/pull/37710#discussion_r978300187.
   
   I think this comment basically is to keep the code style consistent with the codebase that is (roughly) documented in https://spark.apache.org/contributing.html:
   
   > If you’re not sure about the right style for something, try to follow the style of the existing codebase
   
   So, yes, I think we should do some manual adjustments to be consistent in the current codebase in general, which I believe most of people do in their PRs.
   
   Yes, ideally we should fix the Scala style guides to explicitly mention this.
   
   Yes, `scalafmt` has an issue. There have been several discussions here and there, e.g., https://github.com/apache/spark/pull/35379#issuecomment-1033262392.
   



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978357079


##########
connect/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+import "spark/connect/types.proto";
+
+package spark.connect;
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+// A [[Command]] is an operation that is executed by the server that does not directly consume or
+// produce a relational result.
+message Command {
+  oneof command_type {
+    CreateScalarFunction create_function = 1;
+  }
+}
+
+// Simple message that is used to create a scalar function based on the provided function body.
+//
+// This message is used to register for example a Python UDF in the session catalog by providing
+// the serialized method body.
+message CreateScalarFunction {
+  // Fully qualified name of the function including the catalog / schema names.
+  repeated string parts = 1;
+  FunctionLanguage language = 2;
+  bool temporary = 3;
+  repeated Type argument_types = 4;
+  Type return_type = 5;
+
+  // How the function body is defined:
+  oneof function_definition {
+    // As a raw string serialized:
+    bytes serialized_function = 6;
+    // As a code literal
+    string literal_string = 7;
+  }
+
+  enum FunctionLanguage {
+    FUNCTION_LANGUAGE_UNSPECIFIED = 0;
+    FUNCTION_LANGUAGE_SQL = 1;
+    FUNCTION_LANGUAGE_PYTHON = 2;

Review Comment:
   Yes, absolutely. The Python version is actually already needed [SPARK-40532](https://issues.apache.org/jira/browse/SPARK-40532) to reflect that and will leave a comment in the proto as well.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978296601


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))

Review Comment:
   Could you explain what happens when `session` parameters has different values of configurations (`spark.pyspark.driver.python`, `spark.pyspark.python`) from these environment variables?



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977275603


##########
project/plugins.sbt:
##########
@@ -44,3 +44,5 @@ libraryDependencies += "org.ow2.asm"  % "asm-commons" % "9.3"
 addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.8.3")
 
 addSbtPlugin("com.typesafe.sbt" % "sbt-pom-reader" % "2.2.0")
+
+addSbtPlugin("com.thesamet" % "sbt-protoc" % "1.0.1")

Review Comment:
   What is this for?



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


[GitHub] [spark] tgravescs commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
tgravescs commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1251105175

   I would be ok with merging a minimal working version as long as it doesn't impact many other components and destabilize the builds and other developers activities.  If it doesn't fit this, I think it should either be changed to be decoupled or wait til we think it is.  Based on how its a plugin I would think it would be fairly decoupled but I haven't looked at the code yet.
   
   I think we should ideally give people a few more days to give input to make sure they agree.
   .
   Unfortunately just seeing this so will take a high level look in the next day or so.


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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r974424083


##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.sparkconnect.planner
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar

Review Comment:
   Why do we introduce this import style?



##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.sparkconnect.planner
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType

Review Comment:
   ditto



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r974599749


##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.sparkconnect.command

Review Comment:
   I'll change to connect.



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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1251115196

   Thanks for your feedback. Yes, it's pretty much decoupled, and I believe this doesn't affect anything to other components. Sure, I will leave it out for more days.


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


[GitHub] [spark] LuciferYang commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r982001484


##########
connect/pom.xml:
##########
@@ -0,0 +1,279 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent_2.12</artifactId>
+    <version>3.4.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>spark-connect_2.12</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project Connect</name>
+  <url>https://spark.apache.org/</url>
+  <properties>
+    <sbt.project.name>connect</sbt.project.name>
+    <protobuf.version>3.21.1</protobuf.version>
+    <guava.version>31.0.1-jre</guava.version>
+    <io.grpc.version>1.47.0</io.grpc.version>
+    <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-tags_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>${guava.version}</version>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>failureaccess</artifactId>
+      <version>1.0.1</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${protobuf.version}</version>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty-shaded</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-services</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency> <!-- necessary for Java 9+ -->
+      <groupId>org.apache.tomcat</groupId>
+      <artifactId>annotations-api</artifactId>
+      <version>${tomcat.annotations.api.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+  <build>
+    <!-- Protobuf compilation for Spark Connect -->
+    <extensions>
+      <extension>
+        <groupId>kr.motd.maven</groupId>
+        <artifactId>os-maven-plugin</artifactId>
+        <version>1.6.2</version>
+      </extension>
+    </extensions>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-sources</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/main/scala-${scala.binary.version}</source>
+              </sources>
+            </configuration>
+          </execution>
+          <execution>
+            <id>add-scala-test-sources</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>add-test-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/test/gen-java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>

Review Comment:
   @HyukjinKwon @grundprinzip Report another issue:
   
   Compile `connect` module on CentOS release 6.3, the default glibc version is 2.12, this will cause compilation to fail as follows:
   
   ```
   [ERROR] PROTOC FAILED: /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe: /lib64/libc.so.6: version `GLIBC_2.14' not found (required by /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe)
   /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe: /usr/lib64/libstdc++.so.6: version `GLIBCXX_3.4.18' not found (required by /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe)
   /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe: /usr/lib64/libstdc++.so.6: version `GLIBCXX_3.4.14' not found (required by /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe)
   /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe: /usr/lib64/libstdc++.so.6: version `CXXABI_1.3.5' not found (required by /home/disk0/spark-source/connect/target/protoc-plugins/protoc-3.21.1-linux-x86_64.exe) 
   ```
   Already file a jira SPARK-40593,  I think at least we should explicitly point out the compilation dependency somewhere
   
   
   
   
   
   



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977633804


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    // Preconditions.checkState(v.userContext.nonEmpty, "User Context must be present")
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+
+    // TODO empty results (except limit 0) will not yield a schema.

Review Comment:
   This was a left over from a different implementation. Removed it.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977629253


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")

Review Comment:
   Done, missed pushing.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977622423


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))
+
+  def process(): Unit = {
+    command.getCommandTypeCase match {
+      case proto.Command.CommandTypeCase.CREATE_FUNCTION =>
+        handleCreateScalarFunction(command.getCreateFunction)
+      case _ => throw new UnsupportedOperationException(s"${command} not supported.")
+    }
+  }
+
+  // This is a helper function that registers a new Python function in the
+  // [[SparkSession]].

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978427835


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))

Review Comment:
   PTAL and reopen if it is unclear.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978311533


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}

Review Comment:
   ditto. Import style.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978298391


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))
+
+  def process(): Unit = {
+    command.getCommandTypeCase match {
+      case proto.Command.CommandTypeCase.CREATE_FUNCTION =>
+        handleCreateScalarFunction(command.getCreateFunction)
+      case _ => throw new UnsupportedOperationException(s"${command} not supported.")

Review Comment:
   nit. `${command}` -> `$command`.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978301163


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}

Review Comment:
   I reopened the thread, @grundprinzip and @HyukjinKwon , because Apache Spark doesn't use this pattern yet. This PR is trying to introduce this style.
   - https://github.com/apache/spark/pull/37710#discussion_r978300187
   
   ```
   % git grep '^import' | grep '.{$'
   %
   ```



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r976549039


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   Yeah it should be 3.4



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


[GitHub] [spark] grundprinzip commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1252636546

   @pan3793 thanks for the thorough review. I will address the comments shortly.


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


[GitHub] [spark] pan3793 commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975588478


##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <argLine>-ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true</argLine>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala-${scala.binary.version}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-scala-test-sources</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/gen-java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>0.6.1</version>
+                <configuration>
+                    <protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+                    <pluginId>grpc-java</pluginId>
+                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:${io.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+                    <protoSourceRoot>src/main/protobuf</protoSourceRoot>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>compile</goal>
+                            <goal>compile-custom</goal>
+                            <goal>test-compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Shade all GRPC / Guava / Protobuf depencies of this build -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <configuration>
+                    <shadedArtifactAttached>false</shadedArtifactAttached>
+                    <artifactSet>
+                        <includes>
+                            <include>com.google.guava:*</include>
+                            <include>io.grpc:*:</include>
+                            <include>com.google.protobuf:*</include>
+                        </includes>
+                    </artifactSet>
+                    <relocations>
+                        <relocation>
+                            <pattern>com.google.common</pattern>
+                            <shadedPattern>${spark.shade.packageName}.guava</shadedPattern>
+                            <includes>
+                                <include>com.google.common.**</include>
+                            </includes>
+                        </relocation>
+                        <relocation>
+                            <pattern>com.google.protobuf</pattern>
+                            <shadedPattern>${spark.shade.packageName}.protobuf</shadedPattern>

Review Comment:
   how about using `${spark.shade.packageName}.connect.protobuf` to avoid overwriting `spark.shade.packageName` property?



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977639594


##########
project/SparkBuild.scala:
##########
@@ -593,6 +608,60 @@ object Core {
   )
 }
 
+
+object SparkConnect {
+
+  import BuildCommons.protoVersion
+
+  private val shadePrefix = "org.sparkproject.connect"
+  val shadeJar = taskKey[Unit]("Shade the Jars")
+
+  lazy val settings = Seq(
+    // Setting version for the protobuf compiler. This has to be propagated to every sub-project
+    // even if the project is not using it.
+    PB.protocVersion := BuildCommons.protoVersion,
+
+    // For some reason the resolution from the imported Maven build does not work for some
+    // of these dependendencies that we need to shade later on.
+    libraryDependencies ++= Seq(
+      "io.grpc"          % "protoc-gen-grpc-java" % BuildCommons.gprcVersion asProtocPlugin(),
+      "org.scala-lang" % "scala-library" % "2.12.16" % "provided",
+      "com.google.guava" % "guava"                % "31.0.1-jre",
+      "com.google.guava" % "failureaccess"        % "1.0.1",
+      "com.google.protobuf" % "protobuf-java"        % protoVersion % "protobuf"
+    ),
+
+    dependencyOverrides ++= Seq(
+      "com.google.guava" % "guava"                % "31.0.1-jre",
+      "com.google.guava" % "failureaccess"        % "1.0.1",
+      "com.google.protobuf" % "protobuf-java"        % protoVersion
+    ),
+
+    (Compile / PB.targets) := Seq(
+      PB.gens.java                -> (Compile / sourceManaged).value,
+      PB.gens.plugin("grpc-java") -> (Compile / sourceManaged).value
+    ),
+
+    (assembly / test) := false,
+
+    (assembly / logLevel) := Level.Info,
+
+    (assembly / assemblyShadeRules) := Seq(
+      ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.grpc.@0").inAll,
+      ShadeRule.rename("com.google.common.**"-> "org.sparkproject.connect.guava.@1").inAll,
+      ShadeRule.rename("com.google.thirdparty.**"-> "org.sparkproject.connect.guava.@1").inAll,
+      ShadeRule.rename("com.google.protobuf.**"-> "org.sparkproject.connect.protobuf.@1").inAll,

Review Comment:
   Done.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978321735


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Unstable
+class SparkConnectPlugin extends SparkPlugin {
+
+  /**
+   * Return the plugin's driver-side component.
+   *
+   * @return
+   *   The driver-side component, or null if one is not needed.

Review Comment:
   BTW, this method never run `null`. So, we need to remove `, or null if one is not needed` part.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978318196


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _

Review Comment:
   Is this used outside? Maybe, `private var`?



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977644784


##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column
+        ...
+
+    @property
+    def columns(self) -> List[str]:
+        """Returns the list of columns of the current data frame."""
+        if self._plan is None:
+            return []
+        if "columns" not in self._cache and self._plan is not None:
+            pdd = self.limit(0).collect()
+            # Translate to standard pytho array
+            self._cache["columns"] = pdd.columns.values
+        return self._cache["columns"]
+
+    def count(self):
+        """Returns the number of rows in the data frame"""
+        return self.agg([(LiteralExpression(1), "count")]).collect().iloc[0, 0]
+
+    def crossJoin(self, other):
+        ...
+
+    def coalesce(self, num_partitions: int) -> "DataFrame":
+        # TODO needs repartition operator for substrait
+        ...
+
+    def describe(self, cols):
+        # TODO needs analyze to filter out the right columns
+        ...
+
+    def distinct(self) -> "DataFrame":
+        """Returns all distinct rows."""
+        all_cols = self.columns()
+        gf = self.groupBy(*all_cols)
+        return gf.agg()
+
+    def drop(self, *cols: ColumnOrString):
+        # TODO Needs analyze to know which columns to drop
+        all_cols = self.columns()
+        dropped = set([c.name() if isinstance(c, ColumnRef) else self[c].name() for c in cols])
+        filter(lambda x: x in dropped, all_cols)
+
+    def filter(self, condition: Expression) -> "DataFrame":
+        return DataFrame.withPlan(
+            plan.Filter(child=self._plan, filter=condition), session=self._session
+        )
+
+    def first(self):
+        return self.head(1)
+
+    def groupBy(self, *cols: ColumnOrString):
+        return GroupingFrame(self, *cols)
+
+    def head(self, n: int):
+        self.limit(n)
+        return self.collect()
+
+    def join(self, other, on, how=None):
+        return DataFrame.withPlan(
+            plan.Join(left=self._plan, right=other._plan, on=on, how=how),
+            session=self._session,
+        )
+
+    def limit(self, n):
+        return DataFrame.withPlan(plan.Limit(child=self._plan, limit=n), session=self._session)
+
+    def sort(self, *cols: ColumnOrName):
+        """Sort by a specific column"""
+        return DataFrame.withPlan(plan.Sort(self._plan, *cols), session=self._session)
+
+    def show(self, n: int, truncate: Optional[Union[bool, int]], vertical: Optional[bool]):
+        ...
+
+    def union(self, other) -> "DataFrame":
+        return self.unionAll(other)
+
+    def unionAll(self, other: "DataFrame") -> "DataFrame":
+        if other._plan is None:
+            raise ValueError("Argument to Union does not contain a valid plan.")
+        return DataFrame.withPlan(plan.UnionAll(self._plan, other._plan), session=self._session)
+
+    def where(self, condition):
+        return self.filter(condition)
+
+    def _get_alias(self):
+        p = self._plan
+        while p is not None:
+            if isinstance(p, plan.Project) and p.alias:
+                return p.alias
+            p = p._child
+        return None

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977646112


##########
python/pyspark/sql/connect/function_builder.py:
##########
@@ -0,0 +1,118 @@
+#
+# 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.
+#
+
+import functools
+from typing import TYPE_CHECKING
+
+import pyspark.sql.types
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    ScalarFunctionExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+def _build(name: str, *args: ExpressionOrString) -> ScalarFunctionExpression:
+    """
+    Simple wrapper function that converts the arguments into the appropriate types.
+    Parameters
+    ----------
+    name Name of the function to be called.
+    args The list of arguments.
+
+    Returns
+    -------
+    :class:`ScalarFunctionExpression`
+    """
+    cols = [x if isinstance(x, Expression) else ColumnRef.from_qualified_name(x) for x in args]
+    return ScalarFunctionExpression(name, *cols)
+
+
+class FunctionBuilder:
+    """This class is used to build arbitrary functions used in expressions"""
+
+    def __getattr__(self, name):
+        def _(*args: ExpressionOrString) -> ScalarFunctionExpression:
+            return _build(name, *args)
+
+        _.__doc__ = f"""Function to apply {name}"""
+        return _
+
+
+functions = FunctionBuilder()
+
+
+class UserDefinedFunction(Expression):
+    """A user defied function is an expresison that has a reference to the actual
+    Python callable attached. During plan generation, the client sends a command to
+    the server to register the UDF before execution. The expression object can be
+    reused and is not attached to a specific execution. If the internal name of
+    the temporary function is set, it is assumed that the registration has already
+    happened."""
+
+    def __init__(self, func, return_type=pyspark.sql.types.StringType(), args=None):
+        super().__init__()
+
+        self._func_ref = func
+        self._return_type = return_type
+        self._args = list(args)
+        self._func_name = None
+
+    def to_plan(self, session: "RemoteSparkSession") -> Expression:
+        # Needs to materialize the UDF to the server
+        # Only do this once per session
+        func_name = session.register_udf(self._func_ref, self._return_type)
+        # Func name is used for the actual reference
+        return _build(func_name, *self._args).to_plan(session)
+
+    def __str__(self):
+        return f"UserDefinedFunction({self._func_name})"
+
+
+def _create_udf(function, return_type):
+    def wrapper(*cols: "ColumnOrString"):
+        return UserDefinedFunction(func=function, return_type=return_type, args=cols)
+
+    return wrapper
+
+
+def udf(function, return_type=pyspark.sql.types.StringType()):
+    """
+    Returns a callable that represents the column ones arguments are applied

Review Comment:
   I fixed the sentence -> "ones" -> "once"



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978309489


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)
+      joinType = plans.Inner,
+      condition = Some(transformExpression(rel.getOn)),
+      hint = logical.JoinHint.NONE)
+  }
+
+  private def transformSort(rel: proto.Sort): LogicalPlan = {
+    assert(rel.getSortFieldsCount > 0, "SortFields must be present.")

Review Comment:
   Could you revise the error message a little because `SortField` is a type and the field name is `sort_fields`?



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978312377


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {

Review Comment:
   Please follow the community indentation style. In this case, `Databricks Scala Style guide`.
   - https://github.com/databricks/scala-style-guide#spacing-and-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-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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978352846


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))

Review Comment:
   The way the UDFs are currently implemented is really very rudimentary and I will update a comment to reflect that. If the Python version diverge, the executor will throw an error.



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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1257127356

   Merged to master.
   
   
   I will follow up and actively work on cleaning up and followup tasks from tomorrow.


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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r976770428


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()

Review Comment:
   Done.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()

Review Comment:
   Done.



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


[GitHub] [spark] amaliujia commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
amaliujia commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1251291583

   @tgravescs I will follow up on the testing plan doc to address your comments. Please feel free to bring up anything in the doc or 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-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


[GitHub] [spark] LuciferYang commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r981102013


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   Run `dev/make-distribution.sh --tgz` and decompress the generated tar ball, I found the jars related to `grpc`  and `protobuf-java-util-3.19.2.jar` are also placed in the jars directory as follows:
   
   ```
   ls -l *grpc* 
   -rw-r--r--  1 yangjie01  staff   256991  9 27 18:03 grpc-api-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff    30593  9 27 18:03 grpc-context-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff   689433  9 27 18:03 grpc-core-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff  9129585  9 27 18:03 grpc-netty-shaded-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff     5115  9 27 18:03 grpc-protobuf-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff     7570  9 27 18:03 grpc-protobuf-lite-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff   838576  9 27 18:03 grpc-services-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff    50879  9 27 18:03 grpc-stub-1.47.0.jar
   ```
   
   should we explicitly exclude them here due to they already shaded
   
   
   



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978348887


##########
connect/src/main/protobuf/spark/connect/base.proto:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/commands.proto";
+import "spark/connect/relations.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";

Review Comment:
   Removed all the go packages. They're not needed atm.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978317800


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;

Review Comment:
   According to the comment, this seems to be designed to be changed in the future. In this case, please use `private type` if possible. If we open once, it's hard to change.
   ```scala
   type SessionCacheKey = String;
   private type SessionCacheKey = 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-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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978350959


##########
connect/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ The main [[Relation]] type. Fundamentally, a relation is a typed container
+ that has exactly one explicit relation type set.
+
+ When adding new relation types, they have to be registered here.
+ */
+message Relation {
+  RelationCommon common = 1;
+  oneof rel_type {
+    Read read = 2;
+    Project project = 3;
+    Filter filter = 4;
+    Join join = 5;
+    Union union = 6;
+    Sort sort = 7;
+    Fetch fetch = 8;
+    Aggregate aggregate = 9;
+    SQL sql = 10;
+
+    Unknown unknown = 999;
+  }
+}
+
+/*
+ * Used for testing purposes only.

Review Comment:
   I homogenized the proto comment style across all files 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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977627150


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO: Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp, Duration, Period.

Review Comment:
   Created: SPARK-40533



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978303198


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.

Review Comment:
   ```
   - protocol buffers literal
   + protocol buffer literals
   ```



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978298001


##########
connect/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+import "spark/connect/types.proto";
+
+package spark.connect;
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+// A [[Command]] is an operation that is executed by the server that does not directly consume or
+// produce a relational result.
+message Command {
+  oneof command_type {
+    CreateScalarFunction create_function = 1;
+  }
+}
+
+// Simple message that is used to create a scalar function based on the provided function body.
+//
+// This message is used to register for example a Python UDF in the session catalog by providing
+// the serialized method body.
+message CreateScalarFunction {
+  // Fully qualified name of the function including the catalog / schema names.
+  repeated string parts = 1;
+  FunctionLanguage language = 2;
+  bool temporary = 3;
+  repeated Type argument_types = 4;
+  Type return_type = 5;
+
+  // How the function body is defined:
+  oneof function_definition {
+    // As a raw string serialized:
+    bytes serialized_function = 6;
+    // As a code literal
+    string literal_string = 7;
+  }
+
+  enum FunctionLanguage {
+    FUNCTION_LANGUAGE_UNSPECIFIED = 0;
+    FUNCTION_LANGUAGE_SQL = 1;
+    FUNCTION_LANGUAGE_PYTHON = 2;

Review Comment:
   May I ask if this is enough? Do we need additional level of information for specific Python Version or Scala Version (like 2.12 / 2.13) additionally later?



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978359822


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978406012


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;

Review Comment:
   Thanks! Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975725801


##########
dev/infra/Dockerfile:
##########
@@ -65,3 +65,6 @@ RUN Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='ht
 
 # See more in SPARK-39735
 ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library"
+
+# Add Python Deps for Spark Connect.
+RUN python3.9 -m pip install grpcio protobuf

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977633071


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   Done, missed pushing.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978364360


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)

Review Comment:
   Ups, that was a mistake. Sorry.



##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978420688


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+    val data = rows.collect().map(x => x.toSeq.mkString("|")).mkString("\n")
+    val bbb = proto.Response.CSVBatch.newBuilder
+      .setRowCount(-1)
+      .setData(textSchema ++ "\n" ++ data)
+      .build()
+    val response = proto.Response.newBuilder().setClientId(clientId).setCsvBatch(bbb).build()
+
+    // Send all the data
+    responseObserver.onNext(response)
+    responseObserver.onNext(sendMetricsToResponse(clientId, rows))
+    responseObserver.onCompleted()
+  }
+
+  def sendMetricsToResponse(clientId: String, rows: DataFrame): Response = {
+    // Send a last batch with the metrics
+    Response
+      .newBuilder()
+      .setClientId(clientId)
+      .setMetrics(MetricGenerator.buildMetrics(rows.queryExecution.executedPlan))
+      .build()
+  }
+
+  def handleCommand(session: SparkSession, request: Request): Unit = {
+    val command = request.getPlan.getCommand
+    val planner = new SparkConnectCommandPlanner(session, command)
+    planner.process()
+    responseObserver.onCompleted()
+

Review Comment:
   done



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+    val data = rows.collect().map(x => x.toSeq.mkString("|")).mkString("\n")
+    val bbb = proto.Response.CSVBatch.newBuilder
+      .setRowCount(-1)
+      .setData(textSchema ++ "\n" ++ data)
+      .build()
+    val response = proto.Response.newBuilder().setClientId(clientId).setCsvBatch(bbb).build()
+
+    // Send all the data
+    responseObserver.onNext(response)
+    responseObserver.onNext(sendMetricsToResponse(clientId, rows))
+    responseObserver.onCompleted()
+  }
+
+  def sendMetricsToResponse(clientId: String, rows: DataFrame): Response = {
+    // Send a last batch with the metrics
+    Response
+      .newBuilder()
+      .setClientId(clientId)
+      .setMetrics(MetricGenerator.buildMetrics(rows.queryExecution.executedPlan))
+      .build()
+  }
+
+  def handleCommand(session: SparkSession, request: Request): Unit = {
+    val command = request.getPlan.getCommand
+    val planner = new SparkConnectCommandPlanner(session, command)
+    planner.process()
+    responseObserver.onCompleted()
+
+  }
+

Review Comment:
   done.



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


[GitHub] [spark] pan3793 commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975585492


##########
connect/src/main/protobuf/google/protobuf/any.proto:
##########
@@ -0,0 +1,155 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option go_package = "github.com/golang/protobuf/ptypes/any";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "AnyProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+
+// `Any` contains an arbitrary serialized protocol buffer message along with a
+// URL that describes the type of the serialized message.
+//
+// Protobuf library provides support to pack/unpack Any values in the form
+// of utility functions or additional generated methods of the Any type.
+//
+// Example 1: Pack and unpack a message in C++.
+//
+//     Foo foo = ...;
+//     Any any;
+//     any.PackFrom(foo);
+//     ...
+//     if (any.UnpackTo(&foo)) {
+//       ...
+//     }
+//
+// Example 2: Pack and unpack a message in Java.
+//
+//     Foo foo = ...;
+//     Any any = Any.pack(foo);
+//     ...
+//     if (any.is(Foo.class)) {
+//       foo = any.unpack(Foo.class);
+//     }
+//
+//  Example 3: Pack and unpack a message in Python.
+//
+//     foo = Foo(...)
+//     any = Any()
+//     any.Pack(foo)
+//     ...
+//     if any.Is(Foo.DESCRIPTOR):
+//       any.Unpack(foo)
+//       ...
+//
+//  Example 4: Pack and unpack a message in Go
+//
+//      foo := &pb.Foo{...}
+//      any, err := ptypes.MarshalAny(foo)
+//      ...
+//      foo := &pb.Foo{}
+//      if err := ptypes.UnmarshalAny(any, foo); err != nil {
+//        ...
+//      }
+//
+// The pack methods provided by protobuf library will by default use
+// 'type.googleapis.com/full.type.name' as the type URL and the unpack
+// methods only use the fully qualified type name after the last '/'
+// in the type URL, for example "foo.bar.com/x/y.z" will yield type
+// name "y.z".
+//
+//
+// JSON
+// ====
+// The JSON representation of an `Any` value uses the regular
+// representation of the deserialized, embedded message, with an
+// additional field `@type` which contains the type URL. Example:
+//
+//     package google.profile;
+//     message Person {
+//       string first_name = 1;
+//       string last_name = 2;
+//     }
+//
+//     {
+//       "@type": "type.googleapis.com/google.profile.Person",
+//       "firstName": <string>,
+//       "lastName": <string>
+//     }
+//
+// If the embedded message type is well-known and has a custom JSON
+// representation, that representation will be embedded adding a field
+// `value` which holds the custom JSON in addition to the `@type`
+// field. Example (for message [google.protobuf.Duration][]):
+//
+//     {
+//       "@type": "type.googleapis.com/google.protobuf.Duration",
+//       "value": "1.212s"
+//     }
+//
+message Any {

Review Comment:
   The generated class `com.google.protobuf.Empty` is already included in `protobuf-java.jar`, why should we include this proto file again?



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


[GitHub] [spark] Yikun commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
Yikun commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1239137323

   You can retrigger the ci, the issue already fix by github. https://www.githubstatus.com/incidents/d181frs643d4


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


[GitHub] [spark] grundprinzip commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1241164342

   I did more investigations on the build failure. It seems that the doc build for Python (and R) is the only place where we actually call the `build/sbt package` target. Most of the other targets execute `Test/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-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


[GitHub] [spark] Yikf commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
Yikf commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r976257784


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()

Review Comment:
   ditto



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        e.printStackTrace()

Review Comment:
   Shall we using log instead of `printStackTrace `,  It will be printed to standard error output and using a large amount of memory space.
   
   



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977629015


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO: Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp, Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO

Review Comment:
   SPARK-40534



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978405412


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable

Review Comment:
   Done



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978363759


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>

Review Comment:
   Added a comment and created SPARK-40546 to track.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978400858


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)
+      joinType = plans.Inner,
+      condition = Some(transformExpression(rel.getOn)),
+      hint = logical.JoinHint.NONE)
+  }
+
+  private def transformSort(rel: proto.Sort): LogicalPlan = {
+    assert(rel.getSortFieldsCount > 0, "SortFields must be present.")

Review Comment:
   Done, thanks.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977651147


##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema
+    def collect(self, session: "RemoteSparkSession" = None, debug: bool = False):
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        if debug:
+            print(plan)
+
+        return plan
+
+    def _i(self, indent) -> str:
+        return " " * indent
+
+    def print(self, indent=0) -> str:
+        ...
+
+    def _repr_html_(self):
+        ...
+
+
+class Read(LogicalPlan):
+    def __init__(self, table_name: str) -> None:
+        super().__init__(None)
+        self.table_name = table_name
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        plan = proto.Relation()
+        plan.read.named_table.parts.extend(self.table_name.split("."))
+        return plan
+
+    def print(self, indent=0) -> str:
+        return f"{self._i(indent)}<Read table_name={self.table_name}>\n"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Read</b><br />
+                table name: {self.table_name}
+            </li>
+        </ul>
+        """
+
+
+class Project(LogicalPlan):
+    """Logical plan object for a projection.
+
+    All input arguments are directly serialized into the corresponding protocol buffer
+    objects. This class only provides very limited error handling and input validation.
+
+    To be compatible with PySpark, we validate that the input arguments are all
+    expressions to be able to serialize them to the server.
+
+    """
+
+    def __init__(self, child: Optional["LogicalPlan"], *columns: ExpressionOrString) -> None:
+        super().__init__(child)
+        self._raw_columns = list(columns)
+        self.alias = None
+        self._verify_expressions()
+
+    def _verify_expressions(self):
+        """Ensures that all input arguments are instances of Expression."""
+        for c in self._raw_columns:
+            if not isinstance(c, Expression):
+                raise InputValidationError(f"Only Expressions can be used for projections: '{c}'.")
+
+    def withAlias(self, alias) -> LogicalPlan:
+        self.alias = alias
+        return self
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        proj_exprs = [
+            c.to_plan(session)
+            if isinstance(c, Expression)
+            else self.unresolved_attr(*cast(str, c).split("."))
+            for c in self._raw_columns
+        ]  # [self.unresolved_attr(*x) for x in self.columns]

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977650177


##########
python/pyspark/sql/connect/functions.py:
##########
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+from pyspark.sql.connect.column import ColumnRef, LiteralExpression
+from pyspark.sql.connect.column import PrimitiveType
+
+

Review Comment:
   SPARK-40538



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978420144


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}

Review Comment:
   done.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978320023


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we

Review Comment:
   `it's` -> `its`



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


[GitHub] [spark] pan3793 commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r981107986


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   same concerns at https://github.com/apache/spark/pull/37710#discussion_r977631501



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


[GitHub] [spark] LuciferYang commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r981102013


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   Run `dev/make-distribution.sh --tgz` and decompress the generated tar ball, I found the jars related to `grpc`  and `-rw-r--r--  1 yangjie01  staff    73386  9 27 18:03 protobuf-java-util-3.19.2.jar` are also placed in the jars directory as follows:
   
   ```
   ls -l *grpc* 
   -rw-r--r--  1 yangjie01  staff   256991  9 27 18:03 grpc-api-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff    30593  9 27 18:03 grpc-context-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff   689433  9 27 18:03 grpc-core-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff  9129585  9 27 18:03 grpc-netty-shaded-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff     5115  9 27 18:03 grpc-protobuf-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff     7570  9 27 18:03 grpc-protobuf-lite-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff   838576  9 27 18:03 grpc-services-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff    50879  9 27 18:03 grpc-stub-1.47.0.jar
   ```
   
   should we explicitly exclude them 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-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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977660209


##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema
+    def collect(self, session: "RemoteSparkSession" = None, debug: bool = False):
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        if debug:
+            print(plan)
+
+        return plan
+
+    def _i(self, indent) -> str:

Review Comment:
   Removed the function.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977654821


##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.data_frame import DataFrame
+from pyspark.sql.connect.plan import Read
+
+
+class DataFrameReader:
+    def __init__(self, client):
+        self._client = client
+
+    def table(self, tableName: str) -> "DataFrame":
+        df = DataFrame.withPlan(Read(tableName), self._client)
+        return df

Review Comment:
   Done.



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


[GitHub] [spark] Yikun commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
Yikun commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1239418757

   not having a deep look but you didn't change any code in [1][2], so this might relate to some change in `pom.xml`, `SparkBuild.scala`.
   
   [1] https://github.com/grundprinzip/spark/runs/8226589025?check_suite_focus=true#step:21:483
   [2] https://github.com/grundprinzip/spark/runs/8226589025?check_suite_focus=true#step:21:6379


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


[GitHub] [spark] Yikun commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
Yikun commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1239246670

   To speed the base image build time, you could move new added python depends in the end of dev/infra/dockerfile when your PR still WIP.


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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1249269885

   I am thinking about merging it without making major changes in this PR if there aren't major issues found, and I myself will take a look for important/urgent items very soon according to the plan described above.
   
   I would like to be transparent here. The frank reasons that I think as above are as follows:
   
   - Multiple people will intensively cowork together for this component but individual works in a different timezone which makes it difficult to work within this @martin-g's branch.
   - Difficult to manage the credibility. The whole size of work would be very huge, and I would like to avoid sharing the same credit with all the coauthors. Different person will sign off and be the author for individual change.
   - I would like to speed up by fully leveraging individual fork's GitHub Actions resources. Currently, @martin-g's GitHub resource here is a bottleneck.
   
   Hope this plan and thought make sense to other committers too. Are you guys okay with this? @dongjoon-hyun @viirya @mridulm @srowen @wangyum @sunchao @huaxingao (derived from SPIP voting)


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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1249261232

   This is ready for a look now.
   
   Since the whole feature and codes would be very large, we (explicitly I, @martin-g, @amaliujia, and @cloud-fan) discussed offline, and decided to propose to split this. This PR is basically the minimal working version  note that most of code lines here were generated from the protobuf.
   
   SPARK-39375 is a parent JIRA, and we described the current action items at this moment.
   More JIRAs will be filed accordingly to the plan below:
   
   ### High-level plan and design:
   
   - [High-Level Design Doc for Spark Connect](https://docs.google.com/document/d/17X6-P5H2522SnE-gF1BVwyildp_PDX8oXD-4l9vqQmA/edit?usp=sharing)
   - [Spark Connect API Testing Plan](https://docs.google.com/document/d/1n6EgS5vcmbwJUs5KGX4PzjKZVcSKd0qf0gLNZ6NFvOE/edit?usp=sharing)
   
   ### Low-level plan:
   
   **Short-term**
   - Extend test coverage for SparkConnectPlanner (right now at 76% line coverage)
   - Extend test coverage for Spark Connect Python client
   - Type annotations for Spark Connect Python client to re-enable mypy
   - Clean-up documentation in PySpark code for Spark Connect
   - Documentation for PySpark in README and doctests
   - Proto validation in server and/or client
   - Validation: 
     - Syntactic -> Parsing
     - Semantic -> Analysis 
   - Alternatively only return error class to clients upon failures.
   - Initial DSL framework for protobuf testing
   - Restructure the build structure to match with other components
     - Maven
     - SBT 
   
   **Long-term**
   - Testing with custom DSL 
   - `LocalRelation`
   - Better error handling for semantic failures
   - Spark and Session configurations
   - Scala Client
   - SBT incremental build and testing environment
   - DataSources
   - UDFs
   - Packaging / Releasing
   
   
   
   


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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978351469


##########
connect/src/main/protobuf/spark/connect/types.proto:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ This message describes the logical [[Type]] of something. It does not carry the value
+ itself but only describes it.
+ */
+message Type {
+  oneof kind {
+    Boolean bool = 1;
+    I8 i8 = 2;
+    I16 i16 = 3;
+    I32 i32 = 5;
+    I64 i64 = 7;
+    FP32 fp32 = 10;
+    FP64 fp64 = 11;
+    String string = 12;
+    Binary binary = 13;
+    Timestamp timestamp = 14;
+    Date date = 16;
+    Time time = 17;
+    IntervalYear interval_year = 19;
+    IntervalDay interval_day = 20;
+    TimestampTZ timestamp_tz = 29;
+    UUID uuid = 32;
+
+    FixedChar fixed_char = 21;
+    VarChar varchar = 22;
+    FixedBinary fixed_binary = 23;
+    Decimal decimal = 24;
+
+    Struct struct = 25;
+    List list = 27;
+    Map map = 28;
+
+    uint32 user_defined_type_reference = 31;
+  }
+
+  enum Nullability {
+    NULLABILITY_UNSPECIFIED = 0;
+    NULLABILITY_NULLABLE = 1;
+    NULLABILITY_REQUIRED = 2;
+  }
+
+  message Boolean {
+    uint32 type_variation_reference = 1;
+    Nullability nullability = 2;
+  }

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977634110


##########
dev/infra/Dockerfile:
##########
@@ -65,3 +65,6 @@ RUN Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='ht
 
 # See more in SPARK-39735
 ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library"
+
+# Add Python Deps for Spark Connect.

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977652216


##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema
+    def collect(self, session: "RemoteSparkSession" = None, debug: bool = False):
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        if debug:
+            print(plan)
+
+        return plan
+
+    def _i(self, indent) -> str:
+        return " " * indent
+
+    def print(self, indent=0) -> str:
+        ...
+
+    def _repr_html_(self):
+        ...
+
+
+class Read(LogicalPlan):
+    def __init__(self, table_name: str) -> None:
+        super().__init__(None)
+        self.table_name = table_name
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        plan = proto.Relation()
+        plan.read.named_table.parts.extend(self.table_name.split("."))
+        return plan
+
+    def print(self, indent=0) -> str:
+        return f"{self._i(indent)}<Read table_name={self.table_name}>\n"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Read</b><br />
+                table name: {self.table_name}
+            </li>
+        </ul>
+        """
+
+
+class Project(LogicalPlan):
+    """Logical plan object for a projection.
+
+    All input arguments are directly serialized into the corresponding protocol buffer
+    objects. This class only provides very limited error handling and input validation.
+
+    To be compatible with PySpark, we validate that the input arguments are all
+    expressions to be able to serialize them to the server.
+
+    """
+
+    def __init__(self, child: Optional["LogicalPlan"], *columns: ExpressionOrString) -> None:
+        super().__init__(child)
+        self._raw_columns = list(columns)
+        self.alias = None
+        self._verify_expressions()
+
+    def _verify_expressions(self):
+        """Ensures that all input arguments are instances of Expression."""
+        for c in self._raw_columns:
+            if not isinstance(c, Expression):
+                raise InputValidationError(f"Only Expressions can be used for projections: '{c}'.")
+
+    def withAlias(self, alias) -> LogicalPlan:
+        self.alias = alias
+        return self
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        proj_exprs = [
+            c.to_plan(session)
+            if isinstance(c, Expression)
+            else self.unresolved_attr(*cast(str, c).split("."))
+            for c in self._raw_columns
+        ]  # [self.unresolved_attr(*x) for x in self.columns]
+        common = proto.RelationCommon()
+        if self.alias is not None:
+            common.alias = self.alias
+
+        plan = proto.Relation()
+        plan.project.input.CopyFrom(self._child.plan(session))
+        plan.project.expressions.extend(proj_exprs)
+        plan.common.CopyFrom(common)
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Project cols={self._raw_columns}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Project</b><br />
+                Columns: {",".join([str(c) for c in self._raw_columns])}
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Filter(LogicalPlan):
+    def __init__(self, child: Optional["LogicalPlan"], filter: Expression) -> None:
+        super().__init__(child)
+        self.filter = filter
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        plan = proto.Relation()
+        plan.filter.input.CopyFrom(self._child.plan(session))
+        plan.filter.condition.CopyFrom(self.filter.to_plan(session))
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Filter filter={self.filter}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Filter</b><br />
+                Condition: {self.filter}
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Limit(LogicalPlan):
+    def __init__(self, child: Optional["LogicalPlan"], limit: int, offset: int = 0) -> None:
+        super().__init__(child)
+        self.limit = limit
+        self.offset = offset
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        plan = proto.Relation()
+        plan.fetch.input.CopyFrom(self._child.plan(session))
+        plan.fetch.limit = self.limit
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Limit limit={self.limit} offset={self.offset}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Limit</b><br />
+                Limit: {self.limit} <br />
+                Offset: {self.offset} <br />
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Sort(LogicalPlan):
+    def __init__(
+        self, child: Optional["LogicalPlan"], *columns: Union[SortOrder, ColumnRef, str]
+    ) -> None:
+        super().__init__(child)
+        self.columns = list(columns)
+
+    def col_to_sort_field(
+        self, col: Union[SortOrder, ColumnRef, str], session: "RemoteSparkSession"
+    ) -> proto.Sort.SortField:
+        if type(col) is SortOrder:
+            so = cast(SortOrder, col)
+            sf = proto.Sort.SortField()
+            sf.expression.CopyFrom(so.ref.to_plan(session))
+            sf.direction = (
+                proto.Sort.SortDirection.SORT_DIRECTION_ASCENDING
+                if so.ascending
+                else proto.Sort.SortDirection.SORT_DIRECTION_DESCENDING
+            )
+            sf.nulls = (
+                proto.Sort.SortNulls.SORT_NULLS_FIRST
+                if not so.nullsLast
+                else proto.Sort.SortNulls.SORT_NULLS_LAST
+            )
+            return sf
+        else:
+            sf = proto.Sort.SortField()
+            # Check string
+            if type(col) is ColumnRef:
+                sf.expression.CopyFrom(cast(ColumnRef, col).to_plan(session))
+            else:
+                sf.expression.CopyFrom(self.unresolved_attr(cast(str, col)))
+            sf.direction = proto.Sort.SortDirection.SORT_DIRECTION_ASCENDING
+            sf.nulls = proto.Sort.SortNulls.SORT_NULLS_LAST
+            return sf
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        plan = proto.Relation()
+        plan.sort.input.CopyFrom(self._child.plan(session))
+        plan.sort.sort_fields.extend([self.col_to_sort_field(x, session) for x in self.columns])
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Sort columns={self.columns}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Sort</b><br />
+                {", ".join([str(c) for c in self.columns])}
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Aggregate(LogicalPlan):
+    MeasuresType = Sequence[Tuple[ExpressionOrString, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(
+        self,
+        child: Optional["LogicalPlan"],
+        grouping_cols: List[ColumnRef],
+        measures: OptMeasuresType,
+    ) -> None:
+        super().__init__(child)
+        self.grouping_cols = grouping_cols
+        self.measures = measures if measures is not None else []
+
+    def _convert_measure(self, m, session: "RemoteSparkSession"):
+        exp, fun = m
+        measure = proto.Aggregate.Measure()
+        measure.function.name = fun
+        if type(exp) is str:
+            measure.function.arguments.append(self.unresolved_attr(exp))
+        else:
+            measure.function.arguments.append(cast(Expression, exp).to_plan(session))
+        return measure
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        groupings = [x.to_plan(session) for x in self.grouping_cols]
+
+        agg = proto.Relation()
+        agg.aggregate.input.CopyFrom(self._child.plan(session))
+        agg.aggregate.measures.extend(
+            list(map(lambda x: self._convert_measure(x, session), self.measures))
+        )
+
+        gs = proto.Aggregate.GroupingSet()
+        gs.aggregate_expressions.extend(groupings)
+        agg.aggregate.grouping_sets.append(gs)
+        return agg
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return (
+            f"{self._i(indent)}<Sort columns={self.grouping_cols}"
+            f"measures={self.measures}>\n{c_buf}"
+        )
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Aggregation</b><br />
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Join(LogicalPlan):
+    def __init__(
+        self,
+        left: Optional["LogicalPlan"],
+        right: "LogicalPlan",
+        on: ColumnOrString,
+        how: proto.Join.JoinType = proto.Join.JoinType.JOIN_TYPE_INNER,
+    ) -> None:
+        super().__init__(left)
+        self.left = cast(LogicalPlan, left)
+        self.right = right
+        self.on = on
+        if how is None:
+            how = proto.Join.JoinType.JOIN_TYPE_INNER
+        self.how = how
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        rel = proto.Relation()
+        rel.join.left.CopyFrom(self.left.plan(session))
+        rel.join.right.CopyFrom(self.right.plan(session))
+        rel.join.on.CopyFrom(self.to_attr_or_expression(self.on, session))
+        return rel
+
+    def print(self, indent=0) -> str:
+        i = self._i(indent)
+        o = self._i(indent + LogicalPlan.INDENT)
+        n = indent + LogicalPlan.INDENT * 2
+        return (
+            f"{i}<Join on={self.on} how={self.how}>\n{o}"
+            f"left=\n{self.left.print(n)}\n{o}right=\n{self.right.print(n)}"
+        )
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Join</b><br />
+                Left: {self.left._repr_html_()}
+                Right: {self.right._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class UnionAll(LogicalPlan):
+    def __init__(self, child: Optional["LogicalPlan"], other: "LogicalPlan") -> None:
+        super().__init__(child)
+        self.other = other
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        rel = proto.Relation()
+        rel.union.inputs.extend([self._child.plan(session), self.other.plan(session)])
+        rel.union.union_type = proto.Union.UnionType.UNION_TYPE_ALL
+
+    def print(self, indent=0) -> str:
+        assert self._child is not None
+        assert self.other is not None
+
+        i = self._i(indent)
+        o = self._i(indent + LogicalPlan.INDENT)
+        n = indent + LogicalPlan.INDENT * 2
+        return (
+            f"{i}UnionAll\n{o}child1=\n{self._child.print(n)}"
+            f"\n{o}child2=\n{self.other.print(n)}"
+        )
+
+    def _repr_html_(self) -> str:
+        assert self._child is not None
+        assert self.other is not None
+
+        return f"""
+        <ul>
+            <li>
+                <b>Union</b><br />
+                Left: {self._child._repr_html_()}
+                Right: {self.other._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Sql(LogicalPlan):

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977656498


##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema

Review Comment:
   Explain and Schema are already part of the MVP in the DataFrame.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r976770848


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   Sorry, I wasn't sure which version to pick. Changed it to 3.4.0.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978305985


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))

Review Comment:
   What happens here when `UnresolvedFunction` `fun` has only one argument? Is it banned before this invocation?



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978362369


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}

Review Comment:
   Just saw https://github.com/apache/spark/pull/37710#discussion_r978300187.
   
   I think this comment basically is to keep the code style consistent with the codebase that is (roughly) documented in https://spark.apache.org/contributing.html:
   
   > If you’re not sure about the right style for something, try to follow the style of the existing codebase
   
   So, yes, I think we should do some manual adjustments to be consistent in the current codebase in general, which I believe most of people do in their PRs.
   
   Yes, ideally we should fix the Scala style guides to explicitly mention this.
   
   Yes, `scalafmt` has an issue. There have been several discussions here and there, e.g., https://github.com/apache/spark/pull/35379#issuecomment-1033262392.
   
   Yes, we should make the build failed but we currently can't do it for all cases for now. Should ideally fix here https://github.com/apache/spark/blob/master/scalastyle-config.xml to catch them.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978404522


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.

Review Comment:
   Done.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.

Review Comment:
   Removed.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977268571


##########
connect/src/main/protobuf/spark/connect/base.proto:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/commands.proto";
+import "spark/connect/relations.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+
+// A [[Plan]] is the structure that carries the runtime information for the execution from the
+// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference
+// to the underlying logical plan or it can be of the [[Command]] type that is used to execute
+// commands on the server.
+message Plan {
+  oneof op_type {
+    Relation root = 1;
+    Command command = 2;
+  }
+}
+
+// A request to be executed by the service.
+message Request {
+  // The client_id is set by the client to be able to collate streaming responses from
+  // different queries.
+  string client_id = 1;
+  // User context
+  UserContext user_context = 2;
+  // The logical plan to be executed / analyzed.
+  Plan plan = 3;
+
+  // User Context is used to refer to one particular user session that is executing
+  // queries in the backend.
+  message UserContext {
+    string user_id = 1;
+    string user_name = 2;
+  }
+}
+
+// The response of a query, can be one or more for each request. Responses belonging to the
+// same input query, carry the same `client_id`.
+message Response {
+  string client_id = 1;
+
+  // Result type
+  oneof result_type {
+    ArrowBatch batch = 2;
+    CSVBatch csv_batch = 3;
+  }
+
+  // Metrics for the query execution. Typically, this field is only present in the last
+  // batch of results and then represent the overall state of the query execution.
+  Metrics metrics = 4;
+
+  // Batch results of metrics.
+  message ArrowBatch {
+    int64 row_count = 1;
+    int64 uncompressed_bytes = 2;
+    int64 compressed_bytes = 3;
+    bytes data = 4;
+    bytes schema = 5;
+  }
+
+  message CSVBatch {

Review Comment:
   I think CSV is more consistent



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977638831


##########
project/SparkBuild.scala:
##########
@@ -593,6 +608,60 @@ object Core {
   )
 }
 
+
+object SparkConnect {
+
+  import BuildCommons.protoVersion
+
+  private val shadePrefix = "org.sparkproject.connect"
+  val shadeJar = taskKey[Unit]("Shade the Jars")
+
+  lazy val settings = Seq(
+    // Setting version for the protobuf compiler. This has to be propagated to every sub-project
+    // even if the project is not using it.
+    PB.protocVersion := BuildCommons.protoVersion,
+
+    // For some reason the resolution from the imported Maven build does not work for some
+    // of these dependendencies that we need to shade later on.
+    libraryDependencies ++= Seq(
+      "io.grpc"          % "protoc-gen-grpc-java" % BuildCommons.gprcVersion asProtocPlugin(),
+      "org.scala-lang" % "scala-library" % "2.12.16" % "provided",
+      "com.google.guava" % "guava"                % "31.0.1-jre",

Review Comment:
   I "re-formatted" this, please let me know if it matches your expectations.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r982961497


##########
connect/pom.xml:
##########
@@ -0,0 +1,279 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent_2.12</artifactId>
+    <version>3.4.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>spark-connect_2.12</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project Connect</name>
+  <url>https://spark.apache.org/</url>
+  <properties>
+    <sbt.project.name>connect</sbt.project.name>
+    <protobuf.version>3.21.1</protobuf.version>
+    <guava.version>31.0.1-jre</guava.version>
+    <io.grpc.version>1.47.0</io.grpc.version>
+    <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-tags_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>${guava.version}</version>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>failureaccess</artifactId>
+      <version>1.0.1</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${protobuf.version}</version>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty-shaded</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-services</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${io.grpc.version}</version>
+    </dependency>
+    <dependency> <!-- necessary for Java 9+ -->
+      <groupId>org.apache.tomcat</groupId>
+      <artifactId>annotations-api</artifactId>
+      <version>${tomcat.annotations.api.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+  <build>
+    <!-- Protobuf compilation for Spark Connect -->
+    <extensions>
+      <extension>
+        <groupId>kr.motd.maven</groupId>
+        <artifactId>os-maven-plugin</artifactId>
+        <version>1.6.2</version>
+      </extension>
+    </extensions>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-sources</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/main/scala-${scala.binary.version}</source>
+              </sources>
+            </configuration>
+          </execution>
+          <execution>
+            <id>add-scala-test-sources</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>add-test-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/test/gen-java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>

Review Comment:
   Thanks @LuciferYang for checking this out.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977836144


##########
connect/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ The main [[Relation]] type. Fundamentally, a relation is a typed container
+ that has exactly one explicit relation type set.
+
+ When adding new relation types, they have to be registered here.
+ */
+message Relation {
+  RelationCommon common = 1;
+  oneof rel_type {
+    Read read = 2;
+    Project project = 3;
+    Filter filter = 4;
+    Join join = 5;
+    Union union = 6;
+    Sort sort = 7;
+    Fetch fetch = 8;
+    Aggregate aggregate = 9;
+    Sql sql = 10;
+
+    Unknown unknown = 999;
+  }
+}
+
+/*
+ * Used for testing purposes only.
+ */
+message Unknown {}
+
+/*
+ Common metadata of all relations.
+ */
+message RelationCommon {
+  string source_info = 1;
+  string alias = 2;
+}
+
+/*
+ Relation that uses a SQL query to generate the output.
+ */
+message Sql {

Review Comment:
   SQL



##########
connect/src/main/protobuf/spark/connect/base.proto:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/commands.proto";
+import "spark/connect/relations.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+
+// A [[Plan]] is the structure that carries the runtime information for the execution from the
+// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference
+// to the underlying logical plan or it can be of the [[Command]] type that is used to execute
+// commands on the server.
+message Plan {
+  oneof op_type {
+    Relation root = 1;
+    Command command = 2;
+  }
+}
+
+// A request to be executed by the service.
+message Request {
+  // The client_id is set by the client to be able to collate streaming responses from
+  // different queries.
+  string client_id = 1;
+  // User context
+  UserContext user_context = 2;
+  // The logical plan to be executed / analyzed.
+  Plan plan = 3;
+
+  // User Context is used to refer to one particular user session that is executing
+  // queries in the backend.
+  message UserContext {
+    string user_id = 1;
+    string user_name = 2;
+  }
+}
+
+// The response of a query, can be one or more for each request. Responses belonging to the
+// same input query, carry the same `client_id`.
+message Response {
+  string client_id = 1;
+
+  // Result type
+  oneof result_type {
+    ArrowBatch batch = 2;
+    CSVBatch csv_batch = 3;
+  }
+
+  // Metrics for the query execution. Typically, this field is only present in the last
+  // batch of results and then represent the overall state of the query execution.
+  Metrics metrics = 4;
+
+  // Batch results of metrics.
+  message ArrowBatch {
+    int64 row_count = 1;
+    int64 uncompressed_bytes = 2;
+    int64 compressed_bytes = 3;
+    bytes data = 4;
+    bytes schema = 5;
+  }
+
+  message CSVBatch {

Review Comment:
   Done



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978292279


##########
connect/src/main/protobuf/spark/connect/base.proto:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/commands.proto";
+import "spark/connect/relations.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";

Review Comment:
   Is this `databricks` typo?



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978316309


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable

Review Comment:
   ```
   @Since("3.4.0")
   ```



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978316442


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.

Review Comment:
   `Satic` -> `Static`



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978314747


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.

Review Comment:
   ```
   - Analyze a plan provide 
   + Analyze a plan to provide
   ```



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977632805


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Experimental
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(martin.grund) Make port number configurable.

Review Comment:
   Done.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Experimental
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(martin.grund) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Experimental

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977637347


##########
dev/tox.ini:
##########
@@ -51,4 +51,6 @@ exclude =
     python/pyspark/worker.pyi,
     python/pyspark/java_gateway.pyi,
     dev/ansible-for-test-node/*,
+    python/pyspark/sql/connect/proto/*,
+    python/venv/*,

Review Comment:
   I removed the venv. The proto is because of the comment above. The venv is what I use for testing and the linter will complain about my local python virtualenv. I removed it.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978407163


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()

Review Comment:
   I added those to constants until we refactor to configuration.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978401192


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO(SPARK-40534)
+      joinType = plans.Inner,
+      condition = Some(transformExpression(rel.getOn)),
+      hint = logical.JoinHint.NONE)
+  }
+
+  private def transformSort(rel: proto.Sort): LogicalPlan = {
+    assert(rel.getSortFieldsCount > 0, "SortFields must be present.")
+    logical.Sort(
+      child = transformRelation(rel.getInput),
+      global = true,
+      order = rel.getSortFieldsList.asScala.map(transformSortOrderExpression).toSeq)
+  }
+
+  private def transformSortOrderExpression(so: proto.Sort.SortField): expressions.SortOrder = {
+    expressions.SortOrder(
+      child = transformUnresolvedExpression(so.getExpression),
+      direction = so.getDirection match {
+        case proto.Sort.SortDirection.SORT_DIRECTION_DESCENDING => expressions.Descending
+        case _ => expressions.Ascending
+      },
+      nullOrdering = so.getNulls match {
+        case proto.Sort.SortNulls.SORT_NULLS_LAST => expressions.NullsLast
+        case _ => expressions.NullsFirst
+      },
+      sameOrderExpressions = Seq.empty)
+  }
+
+  private def transformAggregate(rel: proto.Aggregate): LogicalPlan = {
+    assert(rel.hasInput)
+    assert(rel.getGroupingSetsCount == 1, "Only one grouping set supported")

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977629941


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>

Review Comment:
   Done



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>

Review Comment:
   Done



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977625111


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))
+
+  def process(): Unit = {
+    command.getCommandTypeCase match {
+      case proto.Command.CommandTypeCase.CREATE_FUNCTION =>
+        handleCreateScalarFunction(command.getCreateFunction)
+      case _ => throw new UnsupportedOperationException(s"${command} not supported.")
+    }
+  }
+
+  // This is a helper function that registers a new Python function in the
+  // [[SparkSession]].
+  def handleCreateScalarFunction(cf: proto.CreateScalarFunction): Unit = {
+    val function = SimplePythonFunction(
+      cf.getSerializedFunction.toByteArray,
+      Maps.newHashMap(),
+      Lists.newArrayList(),
+      pythonVersion,
+      "3.9", // TODO This needs to be an actual version.

Review Comment:
   Created SPARK-40532.



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


[GitHub] [spark] HyukjinKwon closed pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon closed pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies 
URL: https://github.com/apache/spark/pull/37710


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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1251149875

   For clarification, I am fine with reverting the whole component if the plan isn't followed 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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r974602945


##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.sparkconnect.planner
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar

Review Comment:
   I'm running this through `scalafmt` and checkstyle and it doesn't complain. What's the best way to solve this automatically?



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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1250524033

   I will start reviewing closely from late tonight or early tomorrow assuming that we're fine with the plan shared above. I created a new component for `CONNECT`, see also https://github.com/apache/spark/pull/37925.


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


[GitHub] [spark] martin-g commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
martin-g commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1249314050

   It wasn't me! :-)
   You meant Martin Grund (@grundprinzip)


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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r981253712


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   Thanks for pointing this out. I am working on this - would make a PR soon.



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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1259519275

   There's an outstanding comment: https://github.com/apache/spark/pull/37710#discussion_r978291019. I am working on this.


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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975782260


##########
dev/deps/spark-deps-hadoop-3-hive-2.3:
##########
@@ -60,10 +62,20 @@ datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar
 datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar
 derby/10.14.2.0//derby-10.14.2.0.jar
 dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar
+error_prone_annotations/2.10.0//error_prone_annotations-2.10.0.jar
+failureaccess/1.0.1//failureaccess-1.0.1.jar
 flatbuffers-java/1.12.0//flatbuffers-java-1.12.0.jar
 gcs-connector/hadoop3-2.2.7/shaded/gcs-connector-hadoop3-2.2.7-shaded.jar
 generex/1.0.2//generex-1.0.2.jar
 gmetric4j/1.0.10//gmetric4j-1.0.10.jar
+grpc-api/1.47.0//grpc-api-1.47.0.jar

Review Comment:
   Yes, the dependencies are shaded, but the `dev/test-depdencies.sh` script fails if there are not listed 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-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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r974611991


##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.sparkconnect.planner
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar

Review Comment:
   Basically, if I run
   
   ```
   import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar}
   ```
   
   through
   
   ```
   ./build/mvn -Pscala-2.12 scalafmt:format -Dscalafmt.skip=false -Dscalafmt.onlyChanged=false -pl connect
   ```
   
   it becomes this.



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


[GitHub] [spark] HyukjinKwon commented on pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1251143797

   There is a testing plan ([Spark Connect API Testing Plan](https://docs.google.com/document/d/1n6EgS5vcmbwJUs5KGX4PzjKZVcSKd0qf0gLNZ6NFvOE/edit?usp=sharing)) that I and @amaliujia will work on it right away after this PR. I promise that I will make sure on the testing coverage - it's a bit difficult to work together in one branch together because of several issues (https://github.com/apache/spark/pull/37710#issuecomment-1249269885).
   
   FWIW, there is more detailed ([High-Level Design Doc for Spark Connect](https://docs.google.com/document/d/17X6-P5H2522SnE-gF1BVwyildp_PDX8oXD-4l9vqQmA/edit?usp=sharing)) 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-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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977631190


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental

Review Comment:
   Done



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


[GitHub] [spark] pan3793 commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977631501


##########
dev/deps/spark-deps-hadoop-3-hive-2.3:
##########
@@ -60,10 +62,20 @@ datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar
 datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar
 derby/10.14.2.0//derby-10.14.2.0.jar
 dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar
+error_prone_annotations/2.10.0//error_prone_annotations-2.10.0.jar
+failureaccess/1.0.1//failureaccess-1.0.1.jar
 flatbuffers-java/1.12.0//flatbuffers-java-1.12.0.jar
 gcs-connector/hadoop3-2.2.7/shaded/gcs-connector-hadoop3-2.2.7-shaded.jar
 generex/1.0.2//generex-1.0.2.jar
 gmetric4j/1.0.10//gmetric4j-1.0.10.jar
+grpc-api/1.47.0//grpc-api-1.47.0.jar

Review Comment:
   I think it's an unexpected result. If we keep these jars into final binary release tgz, and shade them into spark-connect.jar in the meanwhile, then we deliver two copies of those jars.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978294515


##########
connect/src/main/protobuf/spark/connect/types.proto:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ This message describes the logical [[Type]] of something. It does not carry the value
+ itself but only describes it.
+ */
+message Type {
+  oneof kind {
+    Boolean bool = 1;
+    I8 i8 = 2;
+    I16 i16 = 3;
+    I32 i32 = 5;
+    I64 i64 = 7;
+    FP32 fp32 = 10;
+    FP64 fp64 = 11;
+    String string = 12;
+    Binary binary = 13;
+    Timestamp timestamp = 14;
+    Date date = 16;
+    Time time = 17;
+    IntervalYear interval_year = 19;
+    IntervalDay interval_day = 20;
+    TimestampTZ timestamp_tz = 29;
+    UUID uuid = 32;
+
+    FixedChar fixed_char = 21;
+    VarChar varchar = 22;
+    FixedBinary fixed_binary = 23;
+    Decimal decimal = 24;
+
+    Struct struct = 25;
+    List list = 27;
+    Map map = 28;
+
+    uint32 user_defined_type_reference = 31;
+  }
+
+  enum Nullability {
+    NULLABILITY_UNSPECIFIED = 0;
+    NULLABILITY_NULLABLE = 1;
+    NULLABILITY_REQUIRED = 2;
+  }
+
+  message Boolean {
+    uint32 type_variation_reference = 1;
+    Nullability nullability = 2;
+  }

Review Comment:
   nit. New line after this.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978301163


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}

Review Comment:
   I reopened the thread, @grundprinzip and @HyukjinKwon , because Apache Spark don't use this pattern yet.
   - https://github.com/apache/spark/pull/37710#discussion_r978300187
   
   ```
   % git grep '^import' | grep '.{$'
   %
   ```



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977530554


##########
python/pyspark/sql/connect/README.md:
##########
@@ -0,0 +1,34 @@
+
+# [EXPERIMENTAL] Spark Connect
+
+**Spark Connect is a strictly experimental feature and under heavy development.
+All APIs should be considered volatile and should not be used in production.**
+
+This module contains the implementation of Spark Connect which is a logical plan
+facade for the implementation in Spark. Spark Connect is directly integrated into the build
+of Spark. To enable it, you only need to activate the driver plugin for Spark Connect.
+
+
+
+
+## Build
+
+1. Build Spark as usual per the documentation.
+2. Build and package the Spark Connect package
+   ```commandline

Review Comment:
   Seems like we should switch `commandline` to `bash` or `sh` (https://www.rubycoloredglasses.com/2013/04/languages-supported-by-github-flavored-markdown/)



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977648488


##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column
+        ...
+
+    @property
+    def columns(self) -> List[str]:
+        """Returns the list of columns of the current data frame."""
+        if self._plan is None:
+            return []
+        if "columns" not in self._cache and self._plan is not None:
+            pdd = self.limit(0).collect()
+            # Translate to standard pytho array
+            self._cache["columns"] = pdd.columns.values
+        return self._cache["columns"]
+
+    def count(self):
+        """Returns the number of rows in the data frame"""
+        return self.agg([(LiteralExpression(1), "count")]).collect().iloc[0, 0]
+
+    def crossJoin(self, other):
+        ...
+
+    def coalesce(self, num_partitions: int) -> "DataFrame":
+        # TODO needs repartition operator for substrait

Review Comment:
   removed the todo since the implementation is empty.



##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column
+        ...
+
+    @property
+    def columns(self) -> List[str]:
+        """Returns the list of columns of the current data frame."""
+        if self._plan is None:
+            return []
+        if "columns" not in self._cache and self._plan is not None:
+            pdd = self.limit(0).collect()
+            # Translate to standard pytho array
+            self._cache["columns"] = pdd.columns.values
+        return self._cache["columns"]
+
+    def count(self):
+        """Returns the number of rows in the data frame"""
+        return self.agg([(LiteralExpression(1), "count")]).collect().iloc[0, 0]
+
+    def crossJoin(self, other):
+        ...
+
+    def coalesce(self, num_partitions: int) -> "DataFrame":
+        # TODO needs repartition operator for substrait
+        ...
+
+    def describe(self, cols):
+        # TODO needs analyze to filter out the right columns

Review Comment:
   removed the todo since the implementation is empty.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977644181


##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column

Review Comment:
   I removed the implementation, so that it's just empty for 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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978291019


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   May I ask some questions, @grundprinzip and @HyukjinKwon ?
   1. Do we need to embed this in the Apache Spark binary releases inevitably?
   2. Can we publish like `Kafka` or `Avro`?
   3. Can we put this under profile like `hadoop-cloud` at least?



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977631655


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}

Review Comment:
   Done



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Experimental

Review Comment:
   Done.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978306960


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>

Review Comment:
   Are we assuming case-sensitive function name, `funName`, always? If we are not, this `case "gt"` could be a bug.



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


[GitHub] [spark] tgravescs commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
tgravescs commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r976477804


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   are we targetting this for 3.3.1? I would have normally expected a major feature to go into 3.4.0 



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978319014


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()

Review Comment:
   Shall we avoid to use magic numbers?
   - We can define two constant variables for these magic numbers for now.
   - I guess these could require the official Spark Connector configurations someday.



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


[GitHub] [spark] AmplabJenkins commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1231678175

   Can one of the admins verify this patch?


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


[GitHub] [spark] grundprinzip commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1239174573

   I retriggered the  builds let's see if it helps.


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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978315580


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.

Review Comment:
   Shall we avoid the word, `Trivial`? :)



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978322966


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+    val data = rows.collect().map(x => x.toSeq.mkString("|")).mkString("\n")
+    val bbb = proto.Response.CSVBatch.newBuilder
+      .setRowCount(-1)
+      .setData(textSchema ++ "\n" ++ data)
+      .build()
+    val response = proto.Response.newBuilder().setClientId(clientId).setCsvBatch(bbb).build()
+
+    // Send all the data
+    responseObserver.onNext(response)
+    responseObserver.onNext(sendMetricsToResponse(clientId, rows))
+    responseObserver.onCompleted()
+  }
+
+  def sendMetricsToResponse(clientId: String, rows: DataFrame): Response = {
+    // Send a last batch with the metrics
+    Response
+      .newBuilder()
+      .setClientId(clientId)
+      .setMetrics(MetricGenerator.buildMetrics(rows.queryExecution.executedPlan))
+      .build()
+  }
+
+  def handleCommand(session: SparkSession, request: Request): Unit = {
+    val command = request.getPlan.getCommand
+    val planner = new SparkConnectCommandPlanner(session, command)
+    planner.process()
+    responseObserver.onCompleted()
+

Review Comment:
   nit. Redundant empty line



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+    val data = rows.collect().map(x => x.toSeq.mkString("|")).mkString("\n")
+    val bbb = proto.Response.CSVBatch.newBuilder
+      .setRowCount(-1)
+      .setData(textSchema ++ "\n" ++ data)
+      .build()
+    val response = proto.Response.newBuilder().setClientId(clientId).setCsvBatch(bbb).build()
+
+    // Send all the data
+    responseObserver.onNext(response)
+    responseObserver.onNext(sendMetricsToResponse(clientId, rows))
+    responseObserver.onCompleted()
+  }
+
+  def sendMetricsToResponse(clientId: String, rows: DataFrame): Response = {
+    // Send a last batch with the metrics
+    Response
+      .newBuilder()
+      .setClientId(clientId)
+      .setMetrics(MetricGenerator.buildMetrics(rows.queryExecution.executedPlan))
+      .build()
+  }
+
+  def handleCommand(session: SparkSession, request: Request): Unit = {
+    val command = request.getPlan.getCommand
+    val planner = new SparkConnectCommandPlanner(session, command)
+    planner.process()
+    responseObserver.onCompleted()
+
+  }
+

Review Comment:
   ditto. Redundant empty line



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978405726


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978404056


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {

Review Comment:
   done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977643521


##########
python/pyspark/sql/connect/column.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+from typing import List, Union, cast, get_args, TYPE_CHECKING
+
+import pyspark.sql.connect.proto as proto
+
+PrimitiveType = Union[str, int, bool, float]
+ExpressionOrString = Union[str, "Expression"]
+ColumnOrString = Union[str, "ColumnRef"]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+    import pyspark.sql.connect.proto as proto
+
+
+class Expression(object):
+    """
+    Expression base class.
+    """
+
+    def __init__(self) -> None:  # type: ignore[name-defined]
+        pass
+
+    def to_plan(self, session: "RemoteSparkSession") -> "proto.Expression":  # type: ignore
+        ...
+
+    def __str__(self) -> str:
+        ...
+
+
+class LiteralExpression(Expression):
+    """A literal expression.
+
+    The Python types are converted best effort into the relevant proto types. On the Spark Connect
+    server side, the proto types are converted to the Catalyst equivalents."""
+
+    def __init__(self, value: PrimitiveType) -> None:  # type: ignore[name-defined]
+        super().__init__()
+        self._value = value
+
+    def to_plan(self, session: "RemoteSparkSession") -> "proto.Expression":
+        """Converts the literal expression to the literal in proto.
+
+        TODO This method always assumes the largest type and can thus

Review Comment:
   Done. SPARK-40533



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978322276


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Unstable
+class SparkConnectPlugin extends SparkPlugin {

Review Comment:
   Please create a new file `SparkConnectorPlugin.scala`. This looks worth to have it.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978322440


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}

Review Comment:
   ditto. Import style.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978343068


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}

Review Comment:
   I'm happy to change this in any way you want but can you please make your comment a bit more actionable?
   
   - scalafmt produces an output that is validated in our style checks.
   - I couldn't find anything in the Spark style guide on this topic that would ban an import like this.
   
   Now my question is how do you want the output to look like and how do I produce the output automatically? It would be weird to request manual style adjustments that we don't verify in the build? (In particular since this is new code?)
   
   Thanks in advance!



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978361286


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO(SPARK-40533): Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp,
+   *   Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))

Review Comment:
   I added a temporary assert here. This will be refactored as part of [SPARK-40449](https://issues.apache.org/jira/browse/SPARK-40449)



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978358455


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977629581


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.

Review Comment:
   Done.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978294086


##########
connect/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ The main [[Relation]] type. Fundamentally, a relation is a typed container
+ that has exactly one explicit relation type set.
+
+ When adding new relation types, they have to be registered here.
+ */
+message Relation {
+  RelationCommon common = 1;
+  oneof rel_type {
+    Read read = 2;
+    Project project = 3;
+    Filter filter = 4;
+    Join join = 5;
+    Union union = 6;
+    Sort sort = 7;
+    Fetch fetch = 8;
+    Aggregate aggregate = 9;
+    SQL sql = 10;
+
+    Unknown unknown = 999;
+  }
+}
+
+/*
+ * Used for testing purposes only.

Review Comment:
   Although this is a comment style, can we have a consistent style with the other places like line 57?
   ```
   /*
    Common metadata of all relations.
    */
   ```



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978292279


##########
connect/src/main/protobuf/spark/connect/base.proto:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/commands.proto";
+import "spark/connect/relations.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";

Review Comment:
   Is this `databricks` typo? :)



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977636376


##########
dev/tox.ini:
##########
@@ -51,4 +51,6 @@ exclude =
     python/pyspark/worker.pyi,
     python/pyspark/java_gateway.pyi,
     dev/ansible-for-test-node/*,
+    python/pyspark/sql/connect/proto/*,

Review Comment:
   This is generated code that we cannot re-enable.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977641208


##########
python/mypy.ini:
##########
@@ -23,6 +23,16 @@ show_error_codes = True
 warn_unused_ignores = True
 warn_redundant_casts = True
 
+[mypy-pyspark.sql.connect.*]

Review Comment:
   Done.



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


[GitHub] [spark] LuciferYang commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r981102013


##########
assembly/pom.xml:
##########
@@ -74,6 +74,11 @@
       <artifactId>spark-repl_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-connect_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>

Review Comment:
   Run `dev/make-distribution.sh --tgz` and decompress the generated tar ball, I found the jars related to `grpc`  and `protobuf-java-util-3.19.2.jar` are also placed in the jars directory as follows:
   
   ```
   ls -l *grpc* 
   -rw-r--r--  1 yangjie01  staff   256991  9 27 18:03 grpc-api-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff    30593  9 27 18:03 grpc-context-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff   689433  9 27 18:03 grpc-core-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff  9129585  9 27 18:03 grpc-netty-shaded-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff     5115  9 27 18:03 grpc-protobuf-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff     7570  9 27 18:03 grpc-protobuf-lite-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff   838576  9 27 18:03 grpc-services-1.47.0.jar
   -rw-r--r--  1 yangjie01  staff    50879  9 27 18:03 grpc-stub-1.47.0.jar
   ```
   
   should we explicitly exclude them 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-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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977269118


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   Seems like the change was mistakenly not pushed.



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


[GitHub] [spark] pan3793 commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975579403


##########
.github/workflows/build_and_test.yml:
##########
@@ -536,13 +538,6 @@ jobs:
         key: docs-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }}
         restore-keys: |
           docs-coursier-
-    - name: Cache Maven local repository

Review Comment:
   why remove this step?



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r975731244


##########
project/SparkBuild.scala:
##########
@@ -753,6 +815,7 @@ object OldDeps {
   }
 
   def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq(
+    PB.protocVersion := "3.21.1",

Review Comment:
   Generalized the version into a variable.



##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.sparkconnect.command
+
+import com.google.common.collect.{Lists, Maps}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.{proto => proto}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {

Review Comment:
   Done, added `@Since("3.3.1")`, is this correct?



##########
project/SparkBuild.scala:
##########
@@ -357,7 +366,10 @@ object SparkBuild extends PomBuild {
 
     // To prevent intermittent compilation failures, see also SPARK-33297
     // Apparently we can remove this when we use JDK 11.
-    Test / classLoaderLayeringStrategy := ClassLoaderLayeringStrategy.Flat
+    Test / classLoaderLayeringStrategy := ClassLoaderLayeringStrategy.Flat,
+
+    // BUG fuck me

Review Comment:
   Done. The SBT build was a major pain. Sorry for the leftover.



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>

Review Comment:
   ack will remove.



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>

Review Comment:
   Done.



##########
connect/src/main/buf.work.yaml:
##########
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+version: v1
+directories:
+  - protobuf

Review Comment:
   Done.



##########
project/SparkBuild.scala:
##########
@@ -79,6 +83,11 @@ object BuildCommons {
   val testTempDir = s"$sparkHome/target/tmp"
 
   val javaVersion = settingKey[String]("source and target JVM version for javac and scalac")
+
+  // Google Protobuf version used for generating the protobuf.
+  val protoVersion = "3.21.0"

Review Comment:
   Fixed the version.



##########
connect/src/main/protobuf/google/protobuf/any.proto:
##########
@@ -0,0 +1,155 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option go_package = "github.com/golang/protobuf/ptypes/any";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "AnyProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+
+// `Any` contains an arbitrary serialized protocol buffer message along with a
+// URL that describes the type of the serialized message.
+//
+// Protobuf library provides support to pack/unpack Any values in the form
+// of utility functions or additional generated methods of the Any type.
+//
+// Example 1: Pack and unpack a message in C++.
+//
+//     Foo foo = ...;
+//     Any any;
+//     any.PackFrom(foo);
+//     ...
+//     if (any.UnpackTo(&foo)) {
+//       ...
+//     }
+//
+// Example 2: Pack and unpack a message in Java.
+//
+//     Foo foo = ...;
+//     Any any = Any.pack(foo);
+//     ...
+//     if (any.is(Foo.class)) {
+//       foo = any.unpack(Foo.class);
+//     }
+//
+//  Example 3: Pack and unpack a message in Python.
+//
+//     foo = Foo(...)
+//     any = Any()
+//     any.Pack(foo)
+//     ...
+//     if any.Is(Foo.DESCRIPTOR):
+//       any.Unpack(foo)
+//       ...
+//
+//  Example 4: Pack and unpack a message in Go
+//
+//      foo := &pb.Foo{...}
+//      any, err := ptypes.MarshalAny(foo)
+//      ...
+//      foo := &pb.Foo{}
+//      if err := ptypes.UnmarshalAny(any, foo); err != nil {
+//        ...
+//      }
+//
+// The pack methods provided by protobuf library will by default use
+// 'type.googleapis.com/full.type.name' as the type URL and the unpack
+// methods only use the fully qualified type name after the last '/'
+// in the type URL, for example "foo.bar.com/x/y.z" will yield type
+// name "y.z".
+//
+//
+// JSON
+// ====
+// The JSON representation of an `Any` value uses the regular
+// representation of the deserialized, embedded message, with an
+// additional field `@type` which contains the type URL. Example:
+//
+//     package google.profile;
+//     message Person {
+//       string first_name = 1;
+//       string last_name = 2;
+//     }
+//
+//     {
+//       "@type": "type.googleapis.com/google.profile.Person",
+//       "firstName": <string>,
+//       "lastName": <string>
+//     }
+//
+// If the embedded message type is well-known and has a custom JSON
+// representation, that representation will be embedded adding a field
+// `value` which holds the custom JSON in addition to the `@type`
+// field. Example (for message [google.protobuf.Duration][]):
+//
+//     {
+//       "@type": "type.googleapis.com/google.protobuf.Duration",
+//       "value": "1.212s"
+//     }
+//
+message Any {

Review Comment:
   Correct, I was missing the `protobuf-java` compile dependency. I added it as a compile dependency and it's removed now.



##########
python/mypy.ini:
##########
@@ -110,6 +120,7 @@ strict_optional = False
 [mypy-pyspark.cloudpickle.*]
 ignore_errors = True
 
+

Review Comment:
   Done



##########
project/SparkBuild.scala:
##########
@@ -474,9 +488,11 @@ object SparkBuild extends PomBuild {
 
     sparkSql := {
       (Compile / runMain).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value
-    }
+    },
+

Review Comment:
   Done



##########
connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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

Review Comment:
   Using the package `org.apache.spark.sql.connect.planner`



##########
project/SparkBuild.scala:
##########
@@ -474,9 +488,11 @@ object SparkBuild extends PomBuild {
 
     sparkSql := {
       (Compile / runMain).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value
-    }
+    },
+
   ))(assembly)
 
+

Review Comment:
   Done



##########
python/mypy.ini:
##########
@@ -138,3 +149,10 @@ ignore_missing_imports = True
 
 [mypy-tabulate.*]
 ignore_missing_imports = True
+
+[mypy-google.protobuf.*]
+ignore_missing_imports = True
+
+; Ignore errors for proto generated code
+[mypy-pyspark.sql.connect.proto.*, pyspark.sql.connect.proto]
+ignore_errors = True

Review Comment:
   Done.



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>

Review Comment:
   Done



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>

Review Comment:
   Done.



##########
python/pyspark/sql/connect/README.md:
##########
@@ -0,0 +1,34 @@
+
+# [EXPERIMENTAL] Spark Connect
+
+**Spark Connect is a strictly experimental feature and under heavy development.
+All APIs should be considered volatile and should not be used in production.**
+
+This module contains the implementation of Spark Connect which is a logical plan
+facade for the implementation in Spark. Spark Connect is directly integrated into the build
+of Spark. To enable it, you only need to activate the driver plugin for Spark Connect.
+
+
+
+
+## Build
+
+1. Build Spark as usual per the documentation.
+2. Build and package the Spark Connect package
+   ```commandline
+   ./build/mvn package
+   ```
+   
+## Run Spark Shell
+
+```commandline
+./bin/spark-shell --conf spark.plugins=org.apache.spark.sql.sparkconnect.service.SparkConnectPlugin

Review Comment:
   fixed package name.



##########
connect/src/main/protobuf/google/protobuf/any.proto:
##########
@@ -0,0 +1,155 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option go_package = "github.com/golang/protobuf/ptypes/any";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "AnyProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+
+// `Any` contains an arbitrary serialized protocol buffer message along with a
+// URL that describes the type of the serialized message.
+//
+// Protobuf library provides support to pack/unpack Any values in the form
+// of utility functions or additional generated methods of the Any type.
+//
+// Example 1: Pack and unpack a message in C++.
+//
+//     Foo foo = ...;
+//     Any any;
+//     any.PackFrom(foo);
+//     ...
+//     if (any.UnpackTo(&foo)) {
+//       ...
+//     }
+//
+// Example 2: Pack and unpack a message in Java.
+//
+//     Foo foo = ...;
+//     Any any = Any.pack(foo);
+//     ...
+//     if (any.is(Foo.class)) {
+//       foo = any.unpack(Foo.class);
+//     }
+//
+//  Example 3: Pack and unpack a message in Python.
+//
+//     foo = Foo(...)
+//     any = Any()
+//     any.Pack(foo)
+//     ...
+//     if any.Is(Foo.DESCRIPTOR):
+//       any.Unpack(foo)
+//       ...
+//
+//  Example 4: Pack and unpack a message in Go
+//
+//      foo := &pb.Foo{...}
+//      any, err := ptypes.MarshalAny(foo)
+//      ...
+//      foo := &pb.Foo{}
+//      if err := ptypes.UnmarshalAny(any, foo); err != nil {
+//        ...
+//      }
+//
+// The pack methods provided by protobuf library will by default use
+// 'type.googleapis.com/full.type.name' as the type URL and the unpack
+// methods only use the fully qualified type name after the last '/'
+// in the type URL, for example "foo.bar.com/x/y.z" will yield type
+// name "y.z".
+//
+//
+// JSON
+// ====
+// The JSON representation of an `Any` value uses the regular
+// representation of the deserialized, embedded message, with an
+// additional field `@type` which contains the type URL. Example:
+//
+//     package google.profile;
+//     message Person {
+//       string first_name = 1;
+//       string last_name = 2;
+//     }
+//
+//     {
+//       "@type": "type.googleapis.com/google.profile.Person",
+//       "firstName": <string>,
+//       "lastName": <string>
+//     }
+//
+// If the embedded message type is well-known and has a custom JSON
+// representation, that representation will be embedded adding a field
+// `value` which holds the custom JSON in addition to the `@type`
+// field. Example (for message [google.protobuf.Duration][]):
+//
+//     {
+//       "@type": "type.googleapis.com/google.protobuf.Duration",
+//       "value": "1.212s"
+//     }
+//
+message Any {
+  // A URL/resource name that uniquely identifies the type of the serialized
+  // protocol buffer message. This string must contain at least
+  // one "/" character. The last segment of the URL's path must represent
+  // the fully qualified name of the type (as in
+  // `path/google.protobuf.Duration`). The name should be in a canonical form
+  // (e.g., leading "." is not accepted).
+  //
+  // In practice, teams usually precompile into the binary all types that they
+  // expect it to use in the context of Any. However, for URLs which use the
+  // scheme `http`, `https`, or no scheme, one can optionally set up a type
+  // server that maps type URLs to message definitions as follows:
+  //
+  // * If no scheme is provided, `https` is assumed.
+  // * An HTTP GET on the URL must yield a [google.protobuf.Type][]
+  //   value in binary format, or produce an error.
+  // * Applications are allowed to cache lookup results based on the
+  //   URL, or have them precompiled into a binary to avoid any
+  //   lookup. Therefore, binary compatibility needs to be preserved
+  //   on changes to types. (Use versioned type names to manage
+  //   breaking changes.)
+  //
+  // Note: this functionality is not currently available in the official
+  // protobuf release, and it is not used for type URLs beginning with
+  // type.googleapis.com.
+  //
+  // Schemes other than `http`, `https` (or the empty scheme) might be
+  // used with implementation specific semantics.
+  //
+  string type_url = 1;
+
+  // Must be a valid serialized protocol buffer of the above specified type.
+  bytes value = 2;
+}

Review Comment:
   removed.



##########
connect/src/main/protobuf/google/protobuf/empty.proto:
##########
@@ -0,0 +1,52 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option go_package = "github.com/golang/protobuf/ptypes/empty";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "EmptyProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+option cc_enable_arenas = true;
+
+// A generic empty message that you can re-use to avoid defining duplicated
+// empty messages in your APIs. A typical example is to use it as the request
+// or the response type of an API method. For instance:
+//
+//     service Foo {
+//       rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty);
+//     }
+//
+// The JSON representation for `Empty` is empty JSON object `{}`.
+message Empty {}

Review Comment:
   removed the file



##########
connect/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+/*
+ The main [[Relation]] type. Fundamentally, a relation is a typed container
+ that has exactly one explicit relation type set.
+
+ When adding new relation types, they have to be registered here.
+ */
+message Relation {
+  RelationCommon common = 1;
+  oneof rel_type {
+    Read read = 2;
+    Project project = 3;
+    Filter filter = 4;
+    Join join = 5;
+    Union union = 6;
+    Sort sort = 7;
+    Fetch fetch = 8;
+    Aggregate aggregate = 9;
+    Sql sql = 10;
+
+    Unknown unknown = 999;
+  }
+}
+
+/*
+ * Used for testing purposes only.
+ */
+message Unknown {}
+
+/*
+ Common metadata of all relations.
+ */
+message RelationCommon {
+  string source_info = 1;
+  string alias = 2;
+}
+
+/*
+ Relation that uses a SQL query to generate the output.
+ */
+message Sql {
+  string query = 1;
+}
+
+/*
+ Relation that reads from a file / table or other data source. Does not have additional
+ inputs.
+ */
+message Read {
+  oneof read_type {
+    NamedTable named_table = 1;
+  }
+
+  message NamedTable {
+    repeated string parts = 1;
+  }
+}
+
+/*
+ Projection of a bag of expressions for a given input relation.
+
+ The input relation must be specified.
+ The projected expression can be an arbitrary expression.
+ */
+message Project {
+  Relation input = 1;
+  repeated Expression expressions = 3;
+}
+
+/*
+ Relation that applies a boolean expression `condition` on each row of `input` to produce the output result.
+ */
+message Filter {
+  Relation input = 1;
+  Expression condition = 2;
+}
+
+/*
+ Relation of type [[Join]].
+
+ `left` and `right` must be present.
+ */
+message Join {
+  Relation left = 1;
+  Relation right = 2;
+  Expression on = 3;
+  JoinType how = 4;
+
+  enum JoinType {
+    JOIN_TYPE_UNSPECIFIED = 0;
+    JOIN_TYPE_INNER = 1;
+    JOIN_TYPE_OUTER = 2;
+    JOIN_TYPE_LEFT_OUTER = 3;
+    JOIN_TYPE_RIGHT_OUTER = 4;
+    JOIN_TYPE_ANTI = 5;
+  }
+}
+
+/*
+ Relation of type [[Union]], at least one input must be set.
+ */
+message Union {
+  repeated Relation inputs = 1;
+  UnionType union_type = 2;
+
+  enum UnionType {
+    UNION_TYPE_UNSPECIFIED = 0;
+    UNION_TYPE_DISTINCT = 1;
+    UNION_TYPE_ALL = 2;
+  }
+}
+
+/*
+ Relation of type [[Fetch]] that is used to read `limit` / `offset` rows from the input relation.
+ */
+message Fetch {
+  Relation input = 1;
+  int32 limit = 2;
+  int32 offset = 3;
+}
+
+/*
+ Relation of type [[Aggregate]].
+ */
+message Aggregate {
+  Relation input = 1;
+
+  // Grouping sets are used in rollups
+  repeated GroupingSet grouping_sets = 2;
+
+  // Measures
+  repeated Measure measures = 3;
+
+  message GroupingSet {
+    repeated Expression aggregate_expressions = 1;
+  }
+
+  message Measure {
+    AggregateFunction function = 1;
+    // Conditional filter for SUM(x FILTER WHERE x < 10)
+    Expression filter = 2;
+  }
+
+  message AggregateFunction {
+    string name = 1;
+    repeated Expression arguments = 2;
+  }
+}
+
+/*
+ Relation of type [[Sort]].
+ */
+message Sort {
+  Relation input = 1;
+  repeated SortField sort_fields = 2;
+
+  message SortField {
+    Expression expression = 1;
+    SortDirection direction = 2;
+    SortNulls nulls = 3;
+  }
+
+  enum SortDirection {
+    SORT_DIRECTION_UNSPECIFIED = 0;
+    SORT_DIRECTION_ASCENDING = 1;
+    SORT_DIRECTION_DESCENDING = 2;
+  }
+
+  enum SortNulls {
+    SORT_NULLS_UNSPECIFIED = 0;
+    SORT_NULLS_FIRST = 1;
+    SORT_NULLS_LAST = 2;
+  }
+}

Review Comment:
   done



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <argLine>-ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true</argLine>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala-${scala.binary.version}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-scala-test-sources</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/gen-java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>

Review Comment:
   removed this section.



##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.sparkconnect.service
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sparkconnect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.sparkconnect.planner.SparkConnectPlanner
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    // Preconditions.checkState(v.userContext.nonEmpty, "User Context must be present")
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+
+    // TODO empty results (except limit 0) will not yield a schema.
+
+    val data = rows.collect().map(x => x.toSeq.mkString("|")).mkString("\n")
+    val bbb = proto.Response.CSVBatch.newBuilder
+      .setRowCount(-1)
+      .setData(textSchema ++ "\n" ++ data)
+      .build()
+    val response = proto.Response.newBuilder().setClientId(clientId).setCsvBatch(bbb).build()
+
+    // Send all the data
+    responseObserver.onNext(response)
+
+    //    val batches = rows.collectToArrowBatches()

Review Comment:
   Done



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <argLine>-ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true</argLine>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala-${scala.binary.version}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-scala-test-sources</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/gen-java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>0.6.1</version>
+                <configuration>
+                    <protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+                    <pluginId>grpc-java</pluginId>
+                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:${io.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+                    <protoSourceRoot>src/main/protobuf</protoSourceRoot>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>compile</goal>
+                            <goal>compile-custom</goal>
+                            <goal>test-compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Shade all GRPC / Guava / Protobuf depencies of this build -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <configuration>
+                    <shadedArtifactAttached>false</shadedArtifactAttached>
+                    <artifactSet>
+                        <includes>
+                            <include>com.google.guava:*</include>
+                            <include>io.grpc:*:</include>
+                            <include>com.google.protobuf:*</include>
+                        </includes>
+                    </artifactSet>
+                    <relocations>
+                        <relocation>
+                            <pattern>com.google.common</pattern>

Review Comment:
   will relocated as well.



##########
.github/workflows/build_and_test.yml:
##########
@@ -536,13 +538,6 @@ jobs:
         key: docs-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }}
         restore-keys: |
           docs-coursier-
-    - name: Cache Maven local repository

Review Comment:
   Sorry, this is a leftover from an older issue with the doc build. Will revert the change.



##########
connect/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+import "spark/connect/types.proto";
+
+package spark.connect;
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+option go_package = "github.com/databricks/spark-connect/proto";
+
+// A [[Command]] is an operation that is executed by the server that does not directly consume or
+// produce a relational result.
+message Command {
+  oneof command_type {
+    CreateScalarFunction create_function = 1;
+  }
+}
+
+// Simple message that is used to create a scalar function based on the provided function body.
+//
+// This message is used to register for example a Python UDF in the session catalog by providing
+// the serialized method body.
+message CreateScalarFunction {
+  // Fully qualified name of the function including the catalog / schema names.
+  repeated string parts = 1;
+  FunctionLanguage language = 2;
+  bool temporary = 3;
+  repeated Type argument_types = 4;
+  Type return_type = 5;
+
+  // How the function body is defined:
+  oneof function_definition {
+    // As a raw string serialized:
+    bytes serialized_function = 6;
+    // As a code literal
+    string literal_string = 7;
+  }
+
+  enum FunctionLanguage {
+    FUNCTION_LANGUAGE_UNSPECIFIED = 0;
+    FUNCTION_LANGUAGE_SQL = 1;
+    FUNCTION_LANGUAGE_PYTHON = 2;
+    FUNCTION_LANGUAGE_SCALA = 3;
+  }
+}

Review Comment:
   Done



##########
connect/pom.xml:
##########
@@ -0,0 +1,281 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.4.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>spark-connect_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>Spark Project Connect</name>
+    <url>https://spark.apache.org/</url>
+    <properties>
+        <!-- Package to use when relocating shaded classes. -->
+        <spark.shade.packageName>org.sparkproject.connect</spark.shade.packageName>
+
+        <sbt.project.name>connect</sbt.project.name>
+        <protobuf.version>3.21.1</protobuf.version>
+        <guava.version>31.0.1-jre</guava.version>
+        <io.grpc.version>1.47.0</io.grpc.version>
+        <tomcat.annotations.api.version>6.0.53</tomcat.annotations.api.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- #if scala-2.13 --><!--
+    <dependency>
+      <groupId>org.scala-lang.modules</groupId>
+      <artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
+    </dependency>
+    --><!-- #endif scala-2.13 -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>31.0.1-jre</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>failureaccess</artifactId>
+            <version>1.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-netty-shaded</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-protobuf</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-services</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-stub</artifactId>
+            <version>${io.grpc.version}</version>
+        </dependency>
+        <dependency> <!-- necessary for Java 9+ -->
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>annotations-api</artifactId>
+            <version>${tomcat.annotations.api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    <build>
+        <!-- Protobuf compilation for Spark Connect -->
+        <extensions>
+            <extension>
+                <groupId>kr.motd.maven</groupId>
+                <artifactId>os-maven-plugin</artifactId>
+                <version>1.6.2</version>
+            </extension>
+        </extensions>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+        <plugins>
+
+            <!--
+                 This plugin forces the generation of jar containing sql test classes,
+                 so that the tests classes of external modules can use them. The two execution profiles
+                 are necessary - first one for 'mvn package', second one for 'mvn test-compile'. Ideally,
+                 'mvn compile' should not compile test classes and therefore should not need this.
+                 However, a closed due to "Cannot Reproduce" Maven bug (https://issues.apache.org/jira/browse/MNG-3559)
+                 causes the compilation to fail if catalyst test-jar is not generated. Hence, the
+                 second execution profile for 'mvn test-compile'.
+           -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>prepare-test-jar</id>
+                        <phase>test-compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.scalatest</groupId>
+                <artifactId>scalatest-maven-plugin</artifactId>
+                <configuration>
+                    <argLine>-ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true</argLine>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala-${scala.binary.version}</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-scala-test-sources</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/gen-java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Add protobuf-maven-plugin and provide ScalaPB as a code generation plugin -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.6</source>
+                    <target>1.6</target>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>0.6.1</version>
+                <configuration>
+                    <protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+                    <pluginId>grpc-java</pluginId>
+                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:${io.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+                    <protoSourceRoot>src/main/protobuf</protoSourceRoot>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>compile</goal>
+                            <goal>compile-custom</goal>
+                            <goal>test-compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- Shade all GRPC / Guava / Protobuf depencies of this build -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <configuration>
+                    <shadedArtifactAttached>false</shadedArtifactAttached>
+                    <artifactSet>
+                        <includes>
+                            <include>com.google.guava:*</include>
+                            <include>io.grpc:*:</include>
+                            <include>com.google.protobuf:*</include>
+                        </includes>
+                    </artifactSet>
+                    <relocations>
+                        <relocation>
+                            <pattern>com.google.common</pattern>
+                            <shadedPattern>${spark.shade.packageName}.guava</shadedPattern>
+                            <includes>
+                                <include>com.google.common.**</include>
+                            </includes>
+                        </relocation>
+                        <relocation>
+                            <pattern>com.google.protobuf</pattern>
+                            <shadedPattern>${spark.shade.packageName}.protobuf</shadedPattern>

Review Comment:
   great proposal, thanks. will do.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977655361


##########
python/pyspark/sql/connect/README.md:
##########
@@ -0,0 +1,34 @@
+
+# [EXPERIMENTAL] Spark Connect
+
+**Spark Connect is a strictly experimental feature and under heavy development.
+All APIs should be considered volatile and should not be used in production.**
+
+This module contains the implementation of Spark Connect which is a logical plan
+facade for the implementation in Spark. Spark Connect is directly integrated into the build
+of Spark. To enable it, you only need to activate the driver plugin for Spark Connect.
+
+
+
+
+## Build
+
+1. Build Spark as usual per the documentation.
+2. Build and package the Spark Connect package
+   ```commandline

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977647168


##########
project/SparkBuild.scala:
##########
@@ -1031,12 +1105,13 @@ object Unidoc {
                       Seq (
     publish := {},
 
+

Review Comment:
   Done.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977269368


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))
+
+  def process(): Unit = {
+    command.getCommandTypeCase match {
+      case proto.Command.CommandTypeCase.CREATE_FUNCTION =>
+        handleCreateScalarFunction(command.getCreateFunction)
+      case _ => throw new UnsupportedOperationException(s"${command} not supported.")
+    }
+  }
+
+  // This is a helper function that registers a new Python function in the
+  // [[SparkSession]].

Review Comment:
   ```suggestion
     // `SparkSession`.
   ```
   
   Since this isn't in a Scaladoc



##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO: Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp, Duration, Period.
+   * @param lit
+   * @return
+   *   Expression
+   */
+  private def transformLiteral(lit: proto.Expression.Literal): Expression = {
+    lit.getLiteralTypeCase match {
+      case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => expressions.Literal(lit.getBoolean)
+      case proto.Expression.Literal.LiteralTypeCase.I8 => expressions.Literal(lit.getI8, ByteType)
+      case proto.Expression.Literal.LiteralTypeCase.I16 =>
+        expressions.Literal(lit.getI16, ShortType)
+      case proto.Expression.Literal.LiteralTypeCase.I32 => expressions.Literal(lit.getI32)
+      case proto.Expression.Literal.LiteralTypeCase.I64 => expressions.Literal(lit.getI64)
+      case proto.Expression.Literal.LiteralTypeCase.FP32 =>
+        expressions.Literal(lit.getFp32, FloatType)
+      case proto.Expression.Literal.LiteralTypeCase.FP64 =>
+        expressions.Literal(lit.getFp64, DoubleType)
+      case proto.Expression.Literal.LiteralTypeCase.STRING => expressions.Literal(lit.getString)
+      case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+        expressions.Literal(lit.getBinary, BinaryType)
+      // Microseconds since unix epoch.
+      case proto.Expression.Literal.LiteralTypeCase.TIME =>
+        expressions.Literal(lit.getTime, TimestampType)
+      // Days since UNIX epoch.
+      case proto.Expression.Literal.LiteralTypeCase.DATE =>
+        expressions.Literal(lit.getDate, DateType)
+      case _ => throw InvalidPlanInput("Unsupported Literal Type")
+    }
+  }
+
+  private def transformFetch(limit: proto.Fetch): LogicalPlan = {
+    logical.Limit(
+      child = transformRelation(limit.getInput),
+      limitExpr = expressions.Literal(limit.getLimit, IntegerType))
+  }
+
+  private def lookupFunction(name: String, args: Seq[Expression]): Expression = {
+    UnresolvedFunction(Seq(name), args, isDistinct = false)
+  }
+
+  private def transformScalarFunction(fun: proto.Expression.UnresolvedFunction): Expression = {
+    val funName = fun.getPartsList.asScala.mkString(".")
+    funName match {
+      case "gt" =>
+        expressions.GreaterThan(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case "eq" =>
+        expressions.EqualTo(
+          transformExpression(fun.getArguments(0)),
+          transformExpression(fun.getArguments(1)))
+      case _ =>
+        lookupFunction(funName, fun.getArgumentsList.asScala.map(transformExpression).toSeq)
+    }
+  }
+
+  private def transformUnion(u: proto.Union): LogicalPlan = {
+    assert(u.getInputsCount == 2, "Union must have 2 inputs")
+    val plan = logical.Union(transformRelation(u.getInputs(0)), transformRelation(u.getInputs(1)))
+
+    u.getUnionType match {
+      case proto.Union.UnionType.UNION_TYPE_DISTINCT => logical.Distinct(plan)
+      case proto.Union.UnionType.UNION_TYPE_ALL => plan
+      case _ =>
+        throw InvalidPlanInput(s"Unsupported set operation ${u.getUnionTypeValue}")
+    }
+  }
+
+  private def transformJoin(rel: proto.Join): LogicalPlan = {
+    assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
+    logical.Join(
+      left = transformRelation(rel.getLeft),
+      right = transformRelation(rel.getRight),
+      // TODO

Review Comment:
   Ditto for a JIRA



##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
+
+  def transform(): LogicalPlan = {
+    transformRelation(plan)
+  }
+
+  // The root of the query plan is a relation and we apply the transformations to it.
+  private def transformRelation(rel: proto.Relation): LogicalPlan = {
+    val common = if (rel.hasCommon) {
+      Some(rel.getCommon)
+    } else {
+      None
+    }
+
+    rel.getRelTypeCase match {
+      case proto.Relation.RelTypeCase.READ => transformReadRel(rel.getRead, common)
+      case proto.Relation.RelTypeCase.PROJECT => transformProject(rel.getProject, common)
+      case proto.Relation.RelTypeCase.FILTER => transformFilter(rel.getFilter)
+      case proto.Relation.RelTypeCase.FETCH => transformFetch(rel.getFetch)
+      case proto.Relation.RelTypeCase.JOIN => transformJoin(rel.getJoin)
+      case proto.Relation.RelTypeCase.UNION => transformUnion(rel.getUnion)
+      case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort)
+      case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
+      case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
+      case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
+        throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
+      case _ => throw InvalidPlanInput(s"${rel.getUnknown} not supported.")
+    }
+  }
+
+  private def transformSql(sql: proto.SQL): LogicalPlan = {
+    session.sessionState.sqlParser.parsePlan(sql.getQuery)
+  }
+
+  private def transformReadRel(
+      rel: proto.Read,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRelation = rel.getReadTypeCase match {
+      case proto.Read.ReadTypeCase.NAMED_TABLE =>
+        val child = UnresolvedRelation(rel.getNamedTable.getPartsList.asScala.toSeq)
+        if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+          SubqueryAlias(identifier = common.get.getAlias, child = child)
+        } else {
+          child
+        }
+      case _ => throw InvalidPlanInput()
+    }
+    baseRelation
+  }
+
+  private def transformFilter(rel: proto.Filter): LogicalPlan = {
+    assert(rel.hasInput)
+    val baseRel = transformRelation(rel.getInput)
+    logical.Filter(condition = transformExpression(rel.getCondition), child = baseRel)
+  }
+
+  private def transformProject(
+      rel: proto.Project,
+      common: Option[proto.RelationCommon]): LogicalPlan = {
+    val baseRel = transformRelation(rel.getInput)
+    val projection = if (rel.getExpressionsCount == 0) {
+      Seq(UnresolvedStar(Option.empty))
+    } else {
+      rel.getExpressionsList.asScala.map(transformExpression).map(UnresolvedAlias(_))
+    }
+    val project = logical.Project(projectList = projection.toSeq, child = baseRel)
+    if (common.nonEmpty && common.get.getAlias.nonEmpty) {
+      logical.SubqueryAlias(identifier = common.get.getAlias, child = project)
+    } else {
+      project
+    }
+  }
+
+  private def transformUnresolvedExpression(exp: proto.Expression): UnresolvedAttribute = {
+    UnresolvedAttribute(exp.getUnresolvedAttribute.getPartsList.asScala.toSeq)
+  }
+
+  private def transformExpression(exp: proto.Expression): Expression = {
+    exp.getExprTypeCase match {
+      case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
+        transformUnresolvedExpression(exp)
+      case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
+        transformScalarFunction(exp.getUnresolvedFunction)
+      case _ => throw InvalidPlanInput()
+    }
+  }
+
+  /**
+   * Transforms the protocol buffers literal into the appropriate Catalyst literal expression.
+   *
+   * TODO: Missing support for Instant, BigDecimal, LocalDate, LocalTimestamp, Duration, Period.

Review Comment:
   Would also better have a JIRA.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}

Review Comment:
   ```suggestion
   case class SessionHolder(userId: String, session: SparkSession)
   ```



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    // Preconditions.checkState(v.userContext.nonEmpty, "User Context must be present")

Review Comment:
   Should probably remove this



##########
dev/tox.ini:
##########
@@ -51,4 +51,6 @@ exclude =
     python/pyspark/worker.pyi,
     python/pyspark/java_gateway.pyi,
     dev/ansible-for-test-node/*,
+    python/pyspark/sql/connect/proto/*,
+    python/venv/*,

Review Comment:
   why do we need this?



##########
project/SparkBuild.scala:
##########
@@ -1031,12 +1105,13 @@ object Unidoc {
                       Seq (
     publish := {},
 
+

Review Comment:
   ```suggestion
   ```



##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column

Review Comment:
   JIRA



##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column
+        ...
+
+    @property
+    def columns(self) -> List[str]:
+        """Returns the list of columns of the current data frame."""
+        if self._plan is None:
+            return []
+        if "columns" not in self._cache and self._plan is not None:
+            pdd = self.limit(0).collect()
+            # Translate to standard pytho array
+            self._cache["columns"] = pdd.columns.values
+        return self._cache["columns"]
+
+    def count(self):
+        """Returns the number of rows in the data frame"""
+        return self.agg([(LiteralExpression(1), "count")]).collect().iloc[0, 0]
+
+    def crossJoin(self, other):
+        ...
+
+    def coalesce(self, num_partitions: int) -> "DataFrame":
+        # TODO needs repartition operator for substrait
+        ...
+
+    def describe(self, cols):
+        # TODO needs analyze to filter out the right columns

Review Comment:
   JIRA



##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema
+    def collect(self, session: "RemoteSparkSession" = None, debug: bool = False):
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        if debug:
+            print(plan)
+
+        return plan
+
+    def _i(self, indent) -> str:
+        return " " * indent
+
+    def print(self, indent=0) -> str:
+        ...
+
+    def _repr_html_(self):
+        ...
+
+
+class Read(LogicalPlan):
+    def __init__(self, table_name: str) -> None:
+        super().__init__(None)
+        self.table_name = table_name
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        plan = proto.Relation()
+        plan.read.named_table.parts.extend(self.table_name.split("."))
+        return plan
+
+    def print(self, indent=0) -> str:
+        return f"{self._i(indent)}<Read table_name={self.table_name}>\n"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Read</b><br />
+                table name: {self.table_name}
+            </li>
+        </ul>
+        """
+
+
+class Project(LogicalPlan):
+    """Logical plan object for a projection.
+
+    All input arguments are directly serialized into the corresponding protocol buffer
+    objects. This class only provides very limited error handling and input validation.
+
+    To be compatible with PySpark, we validate that the input arguments are all
+    expressions to be able to serialize them to the server.
+
+    """
+
+    def __init__(self, child: Optional["LogicalPlan"], *columns: ExpressionOrString) -> None:
+        super().__init__(child)
+        self._raw_columns = list(columns)
+        self.alias = None
+        self._verify_expressions()
+
+    def _verify_expressions(self):
+        """Ensures that all input arguments are instances of Expression."""
+        for c in self._raw_columns:
+            if not isinstance(c, Expression):
+                raise InputValidationError(f"Only Expressions can be used for projections: '{c}'.")
+
+    def withAlias(self, alias) -> LogicalPlan:
+        self.alias = alias
+        return self
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        proj_exprs = [
+            c.to_plan(session)
+            if isinstance(c, Expression)
+            else self.unresolved_attr(*cast(str, c).split("."))
+            for c in self._raw_columns
+        ]  # [self.unresolved_attr(*x) for x in self.columns]

Review Comment:
   ```suggestion
           ]
   ```
   
   Maybe remove unused commented codes.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")

Review Comment:
   ditto 3.4.0



##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}
+
+final case class InvalidPlanInput(
+    private val message: String = "",
+    private val cause: Throwable = None.orNull)
+    extends Exception(message, cause)
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   ditto, 3.4.0



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.

Review Comment:
   ```suggestion
    * The SparkConnectService implementation.
   ```



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental

Review Comment:
   I think we should actually use `Unstable` instead of `Experimental`.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Experimental
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(martin.grund) Make port number configurable.

Review Comment:
   Ditto, it should better have a JIRA 
   ```suggestion
      * TODO(SPARK-XXXXX): Make port number configurable.
   ```



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>

Review Comment:
   ```suggestion
         case e: Throwable =>
   ```



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>

Review Comment:
   Maybe ..
   
   ```suggestion
         case e: Throwable =>
   ```
   
   



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Experimental

Review Comment:
   Ditto, `Unstable`



##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema
+    def collect(self, session: "RemoteSparkSession" = None, debug: bool = False):
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        if debug:
+            print(plan)
+
+        return plan
+
+    def _i(self, indent) -> str:

Review Comment:
   What is this? The name `_i` seems difficult to follow.



##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))
+
+  def process(): Unit = {
+    command.getCommandTypeCase match {
+      case proto.Command.CommandTypeCase.CREATE_FUNCTION =>
+        handleCreateScalarFunction(command.getCreateFunction)
+      case _ => throw new UnsupportedOperationException(s"${command} not supported.")
+    }
+  }
+
+  // This is a helper function that registers a new Python function in the
+  // [[SparkSession]].
+  def handleCreateScalarFunction(cf: proto.CreateScalarFunction): Unit = {
+    val function = SimplePythonFunction(
+      cf.getSerializedFunction.toByteArray,
+      Maps.newHashMap(),
+      Lists.newArrayList(),
+      pythonVersion,
+      "3.9", // TODO This needs to be an actual version.

Review Comment:
   Would be better to file a JIRA. e.g.)
   
   ```suggestion
         "3.9", // TODO(SPARK-XXXX): This needs to be an actual version.
   ```



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService Implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Experimental
+@Since("3.3.1")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Exception =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Exception =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Experimental
+case class SessionHolder(userId: String, session: SparkSession) {}
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Experimental
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(martin.grund) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Experimental

Review Comment:
   `Unstable`



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+@Since("3.3.1")

Review Comment:
   ```suggestion
   @Unstable
   @Since("3.4.0")
   ```



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    // Preconditions.checkState(v.userContext.nonEmpty, "User Context must be present")
+    val session =
+      SparkConnectService.getOrCreateIsolatedSession(v.getUserContext.getUserId).session
+    v.getPlan.getOpTypeCase match {
+      case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
+      case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+      case _ =>
+        throw new UnsupportedOperationException(s"${v.getPlan.getOpTypeCase} not supported.")
+    }
+  }
+
+  def handlePlan(session: SparkSession, request: proto.Request): Unit = {
+    // Extract the plan from the request and convert it to a logical plan
+    val planner = new SparkConnectPlanner(request.getPlan.getRoot, session)
+    val rows =
+      Dataset.ofRows(session, planner.transform())
+    processRows(request.getClientId, rows)
+  }
+
+  private def processRows(clientId: String, rows: DataFrame) = {
+    val timeZoneId = SQLConf.get.sessionLocalTimeZone
+    val schema =
+      ByteString.copyFrom(ArrowUtils.toArrowSchema(rows.schema, timeZoneId).toByteArray)
+
+    val textSchema = rows.schema.fields.map(f => f.name).mkString("|")
+
+    // TODO empty results (except limit 0) will not yield a schema.

Review Comment:
   ditto for filing a JIRA



##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema
+    def collect(self, session: "RemoteSparkSession" = None, debug: bool = False):
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        if debug:
+            print(plan)
+
+        return plan
+
+    def _i(self, indent) -> str:
+        return " " * indent
+
+    def print(self, indent=0) -> str:
+        ...
+
+    def _repr_html_(self):
+        ...
+
+
+class Read(LogicalPlan):
+    def __init__(self, table_name: str) -> None:
+        super().__init__(None)
+        self.table_name = table_name
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        plan = proto.Relation()
+        plan.read.named_table.parts.extend(self.table_name.split("."))
+        return plan
+
+    def print(self, indent=0) -> str:
+        return f"{self._i(indent)}<Read table_name={self.table_name}>\n"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Read</b><br />
+                table name: {self.table_name}
+            </li>
+        </ul>
+        """
+
+
+class Project(LogicalPlan):
+    """Logical plan object for a projection.
+
+    All input arguments are directly serialized into the corresponding protocol buffer
+    objects. This class only provides very limited error handling and input validation.
+
+    To be compatible with PySpark, we validate that the input arguments are all
+    expressions to be able to serialize them to the server.
+
+    """
+
+    def __init__(self, child: Optional["LogicalPlan"], *columns: ExpressionOrString) -> None:
+        super().__init__(child)
+        self._raw_columns = list(columns)
+        self.alias = None
+        self._verify_expressions()
+
+    def _verify_expressions(self):
+        """Ensures that all input arguments are instances of Expression."""
+        for c in self._raw_columns:
+            if not isinstance(c, Expression):
+                raise InputValidationError(f"Only Expressions can be used for projections: '{c}'.")
+
+    def withAlias(self, alias) -> LogicalPlan:
+        self.alias = alias
+        return self
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        proj_exprs = [
+            c.to_plan(session)
+            if isinstance(c, Expression)
+            else self.unresolved_attr(*cast(str, c).split("."))
+            for c in self._raw_columns
+        ]  # [self.unresolved_attr(*x) for x in self.columns]
+        common = proto.RelationCommon()
+        if self.alias is not None:
+            common.alias = self.alias
+
+        plan = proto.Relation()
+        plan.project.input.CopyFrom(self._child.plan(session))
+        plan.project.expressions.extend(proj_exprs)
+        plan.common.CopyFrom(common)
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Project cols={self._raw_columns}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Project</b><br />
+                Columns: {",".join([str(c) for c in self._raw_columns])}
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Filter(LogicalPlan):
+    def __init__(self, child: Optional["LogicalPlan"], filter: Expression) -> None:
+        super().__init__(child)
+        self.filter = filter
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        plan = proto.Relation()
+        plan.filter.input.CopyFrom(self._child.plan(session))
+        plan.filter.condition.CopyFrom(self.filter.to_plan(session))
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Filter filter={self.filter}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Filter</b><br />
+                Condition: {self.filter}
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Limit(LogicalPlan):
+    def __init__(self, child: Optional["LogicalPlan"], limit: int, offset: int = 0) -> None:
+        super().__init__(child)
+        self.limit = limit
+        self.offset = offset
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        plan = proto.Relation()
+        plan.fetch.input.CopyFrom(self._child.plan(session))
+        plan.fetch.limit = self.limit
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Limit limit={self.limit} offset={self.offset}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Limit</b><br />
+                Limit: {self.limit} <br />
+                Offset: {self.offset} <br />
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Sort(LogicalPlan):
+    def __init__(
+        self, child: Optional["LogicalPlan"], *columns: Union[SortOrder, ColumnRef, str]
+    ) -> None:
+        super().__init__(child)
+        self.columns = list(columns)
+
+    def col_to_sort_field(
+        self, col: Union[SortOrder, ColumnRef, str], session: "RemoteSparkSession"
+    ) -> proto.Sort.SortField:
+        if type(col) is SortOrder:
+            so = cast(SortOrder, col)
+            sf = proto.Sort.SortField()
+            sf.expression.CopyFrom(so.ref.to_plan(session))
+            sf.direction = (
+                proto.Sort.SortDirection.SORT_DIRECTION_ASCENDING
+                if so.ascending
+                else proto.Sort.SortDirection.SORT_DIRECTION_DESCENDING
+            )
+            sf.nulls = (
+                proto.Sort.SortNulls.SORT_NULLS_FIRST
+                if not so.nullsLast
+                else proto.Sort.SortNulls.SORT_NULLS_LAST
+            )
+            return sf
+        else:
+            sf = proto.Sort.SortField()
+            # Check string
+            if type(col) is ColumnRef:
+                sf.expression.CopyFrom(cast(ColumnRef, col).to_plan(session))
+            else:
+                sf.expression.CopyFrom(self.unresolved_attr(cast(str, col)))
+            sf.direction = proto.Sort.SortDirection.SORT_DIRECTION_ASCENDING
+            sf.nulls = proto.Sort.SortNulls.SORT_NULLS_LAST
+            return sf
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        plan = proto.Relation()
+        plan.sort.input.CopyFrom(self._child.plan(session))
+        plan.sort.sort_fields.extend([self.col_to_sort_field(x, session) for x in self.columns])
+        return plan
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return f"{self._i(indent)}<Sort columns={self.columns}>\n{c_buf}"
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Sort</b><br />
+                {", ".join([str(c) for c in self.columns])}
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Aggregate(LogicalPlan):
+    MeasuresType = Sequence[Tuple[ExpressionOrString, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(
+        self,
+        child: Optional["LogicalPlan"],
+        grouping_cols: List[ColumnRef],
+        measures: OptMeasuresType,
+    ) -> None:
+        super().__init__(child)
+        self.grouping_cols = grouping_cols
+        self.measures = measures if measures is not None else []
+
+    def _convert_measure(self, m, session: "RemoteSparkSession"):
+        exp, fun = m
+        measure = proto.Aggregate.Measure()
+        measure.function.name = fun
+        if type(exp) is str:
+            measure.function.arguments.append(self.unresolved_attr(exp))
+        else:
+            measure.function.arguments.append(cast(Expression, exp).to_plan(session))
+        return measure
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        groupings = [x.to_plan(session) for x in self.grouping_cols]
+
+        agg = proto.Relation()
+        agg.aggregate.input.CopyFrom(self._child.plan(session))
+        agg.aggregate.measures.extend(
+            list(map(lambda x: self._convert_measure(x, session), self.measures))
+        )
+
+        gs = proto.Aggregate.GroupingSet()
+        gs.aggregate_expressions.extend(groupings)
+        agg.aggregate.grouping_sets.append(gs)
+        return agg
+
+    def print(self, indent=0) -> str:
+        c_buf = self._child.print(indent + LogicalPlan.INDENT) if self._child else ""
+        return (
+            f"{self._i(indent)}<Sort columns={self.grouping_cols}"
+            f"measures={self.measures}>\n{c_buf}"
+        )
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Aggregation</b><br />
+                {self._child._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Join(LogicalPlan):
+    def __init__(
+        self,
+        left: Optional["LogicalPlan"],
+        right: "LogicalPlan",
+        on: ColumnOrString,
+        how: proto.Join.JoinType = proto.Join.JoinType.JOIN_TYPE_INNER,
+    ) -> None:
+        super().__init__(left)
+        self.left = cast(LogicalPlan, left)
+        self.right = right
+        self.on = on
+        if how is None:
+            how = proto.Join.JoinType.JOIN_TYPE_INNER
+        self.how = how
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        rel = proto.Relation()
+        rel.join.left.CopyFrom(self.left.plan(session))
+        rel.join.right.CopyFrom(self.right.plan(session))
+        rel.join.on.CopyFrom(self.to_attr_or_expression(self.on, session))
+        return rel
+
+    def print(self, indent=0) -> str:
+        i = self._i(indent)
+        o = self._i(indent + LogicalPlan.INDENT)
+        n = indent + LogicalPlan.INDENT * 2
+        return (
+            f"{i}<Join on={self.on} how={self.how}>\n{o}"
+            f"left=\n{self.left.print(n)}\n{o}right=\n{self.right.print(n)}"
+        )
+
+    def _repr_html_(self):
+        return f"""
+        <ul>
+            <li>
+                <b>Join</b><br />
+                Left: {self.left._repr_html_()}
+                Right: {self.right._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class UnionAll(LogicalPlan):
+    def __init__(self, child: Optional["LogicalPlan"], other: "LogicalPlan") -> None:
+        super().__init__(child)
+        self.other = other
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        assert self._child is not None
+        rel = proto.Relation()
+        rel.union.inputs.extend([self._child.plan(session), self.other.plan(session)])
+        rel.union.union_type = proto.Union.UnionType.UNION_TYPE_ALL
+
+    def print(self, indent=0) -> str:
+        assert self._child is not None
+        assert self.other is not None
+
+        i = self._i(indent)
+        o = self._i(indent + LogicalPlan.INDENT)
+        n = indent + LogicalPlan.INDENT * 2
+        return (
+            f"{i}UnionAll\n{o}child1=\n{self._child.print(n)}"
+            f"\n{o}child2=\n{self.other.print(n)}"
+        )
+
+    def _repr_html_(self) -> str:
+        assert self._child is not None
+        assert self.other is not None
+
+        return f"""
+        <ul>
+            <li>
+                <b>Union</b><br />
+                Left: {self._child._repr_html_()}
+                Right: {self.other._repr_html_()}
+            </li>
+        </uL>
+        """
+
+
+class Sql(LogicalPlan):

Review Comment:
   Maybe `SQL`



##########
dev/infra/Dockerfile:
##########
@@ -65,3 +65,6 @@ RUN Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='ht
 
 # See more in SPARK-39735
 ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library"
+
+# Add Python Deps for Spark Connect.

Review Comment:
   ```suggestion
   # Add Python deps for Spark Connect.
   ```



##########
dev/tox.ini:
##########
@@ -51,4 +51,6 @@ exclude =
     python/pyspark/worker.pyi,
     python/pyspark/java_gateway.pyi,
     dev/ansible-for-test-node/*,
+    python/pyspark/sql/connect/proto/*,

Review Comment:
   Should add a TODO with a JIRA (enabling it)



##########
project/SparkBuild.scala:
##########
@@ -593,6 +608,60 @@ object Core {
   )
 }
 
+
+object SparkConnect {
+
+  import BuildCommons.protoVersion
+
+  private val shadePrefix = "org.sparkproject.connect"
+  val shadeJar = taskKey[Unit]("Shade the Jars")
+
+  lazy val settings = Seq(
+    // Setting version for the protobuf compiler. This has to be propagated to every sub-project
+    // even if the project is not using it.
+    PB.protocVersion := BuildCommons.protoVersion,
+
+    // For some reason the resolution from the imported Maven build does not work for some
+    // of these dependendencies that we need to shade later on.
+    libraryDependencies ++= Seq(
+      "io.grpc"          % "protoc-gen-grpc-java" % BuildCommons.gprcVersion asProtocPlugin(),
+      "org.scala-lang" % "scala-library" % "2.12.16" % "provided",
+      "com.google.guava" % "guava"                % "31.0.1-jre",

Review Comment:
   Would better to have the indentation same as others in this file.



##########
python/mypy.ini:
##########
@@ -23,6 +23,16 @@ show_error_codes = True
 warn_unused_ignores = True
 warn_redundant_casts = True
 
+[mypy-pyspark.sql.connect.*]

Review Comment:
   Would need a JIRA for this too.



##########
python/pyspark/sql/connect/README.md:
##########
@@ -0,0 +1,34 @@
+
+# [EXPERIMENTAL] Spark Connect
+
+**Spark Connect is a strictly experimental feature and under heavy development.
+All APIs should be considered volatile and should not be used in production.**
+
+This module contains the implementation of Spark Connect which is a logical plan
+facade for the implementation in Spark. Spark Connect is directly integrated into the build
+of Spark. To enable it, you only need to activate the driver plugin for Spark Connect.
+
+
+
+
+## Build
+
+1. Build Spark as usual per the documentation.
+2. Build and package the Spark Connect package
+   ```commandline
+   ./build/mvn package

Review Comment:
   Do we have a command for SBT too?



##########
python/pyspark/sql/connect/column.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+from typing import List, Union, cast, get_args, TYPE_CHECKING
+
+import pyspark.sql.connect.proto as proto
+
+PrimitiveType = Union[str, int, bool, float]
+ExpressionOrString = Union[str, "Expression"]
+ColumnOrString = Union[str, "ColumnRef"]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+    import pyspark.sql.connect.proto as proto
+
+
+class Expression(object):
+    """
+    Expression base class.
+    """
+
+    def __init__(self) -> None:  # type: ignore[name-defined]
+        pass
+
+    def to_plan(self, session: "RemoteSparkSession") -> "proto.Expression":  # type: ignore
+        ...
+
+    def __str__(self) -> str:
+        ...
+
+
+class LiteralExpression(Expression):
+    """A literal expression.
+
+    The Python types are converted best effort into the relevant proto types. On the Spark Connect
+    server side, the proto types are converted to the Catalyst equivalents."""
+
+    def __init__(self, value: PrimitiveType) -> None:  # type: ignore[name-defined]
+        super().__init__()
+        self._value = value
+
+    def to_plan(self, session: "RemoteSparkSession") -> "proto.Expression":
+        """Converts the literal expression to the literal in proto.
+
+        TODO This method always assumes the largest type and can thus

Review Comment:
   Would need a JIRA.



##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column
+        ...
+
+    @property
+    def columns(self) -> List[str]:
+        """Returns the list of columns of the current data frame."""
+        if self._plan is None:
+            return []
+        if "columns" not in self._cache and self._plan is not None:
+            pdd = self.limit(0).collect()
+            # Translate to standard pytho array
+            self._cache["columns"] = pdd.columns.values
+        return self._cache["columns"]
+
+    def count(self):
+        """Returns the number of rows in the data frame"""
+        return self.agg([(LiteralExpression(1), "count")]).collect().iloc[0, 0]
+
+    def crossJoin(self, other):
+        ...
+
+    def coalesce(self, num_partitions: int) -> "DataFrame":
+        # TODO needs repartition operator for substrait

Review Comment:
   JIRA



##########
project/plugins.sbt:
##########
@@ -44,3 +44,5 @@ libraryDependencies += "org.ow2.asm"  % "asm-commons" % "9.3"
 addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.8.3")
 
 addSbtPlugin("com.typesafe.sbt" % "sbt-pom-reader" % "2.2.0")
+
+addSbtPlugin("com.thesamet" % "sbt-protoc" % "1.0.1")

Review Comment:
   What is this for?



##########
python/pyspark/sql/connect/function_builder.py:
##########
@@ -0,0 +1,118 @@
+#
+# 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.
+#
+
+import functools
+from typing import TYPE_CHECKING
+
+import pyspark.sql.types
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    ScalarFunctionExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+def _build(name: str, *args: ExpressionOrString) -> ScalarFunctionExpression:
+    """
+    Simple wrapper function that converts the arguments into the appropriate types.
+    Parameters
+    ----------
+    name Name of the function to be called.
+    args The list of arguments.
+
+    Returns
+    -------
+    :class:`ScalarFunctionExpression`
+    """
+    cols = [x if isinstance(x, Expression) else ColumnRef.from_qualified_name(x) for x in args]
+    return ScalarFunctionExpression(name, *cols)
+
+
+class FunctionBuilder:
+    """This class is used to build arbitrary functions used in expressions"""
+
+    def __getattr__(self, name):
+        def _(*args: ExpressionOrString) -> ScalarFunctionExpression:
+            return _build(name, *args)
+
+        _.__doc__ = f"""Function to apply {name}"""
+        return _
+
+
+functions = FunctionBuilder()
+
+
+class UserDefinedFunction(Expression):
+    """A user defied function is an expresison that has a reference to the actual
+    Python callable attached. During plan generation, the client sends a command to
+    the server to register the UDF before execution. The expression object can be
+    reused and is not attached to a specific execution. If the internal name of
+    the temporary function is set, it is assumed that the registration has already
+    happened."""
+
+    def __init__(self, func, return_type=pyspark.sql.types.StringType(), args=None):
+        super().__init__()
+
+        self._func_ref = func
+        self._return_type = return_type
+        self._args = list(args)
+        self._func_name = None
+
+    def to_plan(self, session: "RemoteSparkSession") -> Expression:
+        # Needs to materialize the UDF to the server
+        # Only do this once per session
+        func_name = session.register_udf(self._func_ref, self._return_type)
+        # Func name is used for the actual reference
+        return _build(func_name, *self._args).to_plan(session)
+
+    def __str__(self):
+        return f"UserDefinedFunction({self._func_name})"
+
+
+def _create_udf(function, return_type):
+    def wrapper(*cols: "ColumnOrString"):
+        return UserDefinedFunction(func=function, return_type=return_type, args=cols)
+
+    return wrapper
+
+
+def udf(function, return_type=pyspark.sql.types.StringType()):
+    """
+    Returns a callable that represents the column ones arguments are applied

Review Comment:
   Can we complete the docs or file a JIRA?



##########
project/SparkBuild.scala:
##########
@@ -593,6 +608,60 @@ object Core {
   )
 }
 
+
+object SparkConnect {
+
+  import BuildCommons.protoVersion
+
+  private val shadePrefix = "org.sparkproject.connect"
+  val shadeJar = taskKey[Unit]("Shade the Jars")
+
+  lazy val settings = Seq(
+    // Setting version for the protobuf compiler. This has to be propagated to every sub-project
+    // even if the project is not using it.
+    PB.protocVersion := BuildCommons.protoVersion,
+
+    // For some reason the resolution from the imported Maven build does not work for some
+    // of these dependendencies that we need to shade later on.
+    libraryDependencies ++= Seq(
+      "io.grpc"          % "protoc-gen-grpc-java" % BuildCommons.gprcVersion asProtocPlugin(),
+      "org.scala-lang" % "scala-library" % "2.12.16" % "provided",
+      "com.google.guava" % "guava"                % "31.0.1-jre",
+      "com.google.guava" % "failureaccess"        % "1.0.1",
+      "com.google.protobuf" % "protobuf-java"        % protoVersion % "protobuf"
+    ),
+
+    dependencyOverrides ++= Seq(
+      "com.google.guava" % "guava"                % "31.0.1-jre",
+      "com.google.guava" % "failureaccess"        % "1.0.1",
+      "com.google.protobuf" % "protobuf-java"        % protoVersion
+    ),
+
+    (Compile / PB.targets) := Seq(
+      PB.gens.java                -> (Compile / sourceManaged).value,
+      PB.gens.plugin("grpc-java") -> (Compile / sourceManaged).value
+    ),
+
+    (assembly / test) := false,
+
+    (assembly / logLevel) := Level.Info,
+
+    (assembly / assemblyShadeRules) := Seq(
+      ShadeRule.rename("io.grpc.**" -> "org.sparkproject.connect.grpc.@0").inAll,
+      ShadeRule.rename("com.google.common.**"-> "org.sparkproject.connect.guava.@1").inAll,
+      ShadeRule.rename("com.google.thirdparty.**"-> "org.sparkproject.connect.guava.@1").inAll,
+      ShadeRule.rename("com.google.protobuf.**"-> "org.sparkproject.connect.protobuf.@1").inAll,

Review Comment:
   ```suggestion
         ShadeRule.rename("com.google.common.**" -> "org.sparkproject.connect.guava.@1").inAll,
         ShadeRule.rename("com.google.thirdparty.**" -> "org.sparkproject.connect.guava.@1").inAll,
         ShadeRule.rename("com.google.protobuf.**" -> "org.sparkproject.connect.protobuf.@1").inAll,
   ```



##########
python/pyspark/sql/connect/functions.py:
##########
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+from pyspark.sql.connect.column import ColumnRef, LiteralExpression
+from pyspark.sql.connect.column import PrimitiveType
+
+

Review Comment:
   Can we file a JIRA to complete this functions?



##########
python/pyspark/sql/connect/plan.py:
##########
@@ -0,0 +1,468 @@
+#
+# 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.
+#
+
+from typing import (
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.proto as proto
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    SortOrder,
+)
+
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+class InputValidationError(Exception):
+    pass
+
+
+class LogicalPlan(object):
+
+    INDENT = 2
+
+    def __init__(self, child: Optional["LogicalPlan"]) -> None:
+        self._child = child
+
+    def unresolved_attr(self, *colNames: str) -> proto.Expression:
+        """Creates an unresolved attribute from a column name."""
+        exp = proto.Expression()
+        exp.unresolved_attribute.parts.extend(list(colNames))
+        return exp
+
+    def to_attr_or_expression(
+        self, col: ColumnOrString, session: "RemoteSparkSession"
+    ) -> proto.Expression:
+        """Returns either an instance of an unresolved attribute or the serialized
+        expression value of the column."""
+        if type(col) is str:
+            return self.unresolved_attr(cast(str, col))
+        else:
+            return cast(ColumnRef, col).to_plan(session)
+
+    def plan(self, session: "RemoteSparkSession") -> proto.Relation:
+        ...
+
+    def _verify(self, session: "RemoteSparkSession") -> bool:
+        """This method is used to verify that the current logical plan
+        can be serialized to Proto and back and afterwards is identical."""
+        plan = proto.Plan()
+        plan.root.CopyFrom(self.plan(session))
+
+        serialized_plan = plan.SerializeToString()
+        test_plan = proto.Plan()
+        test_plan.ParseFromString(serialized_plan)
+
+        return test_plan == plan
+
+    # TODO(martin.grund) explain , schema

Review Comment:
   JIRA



##########
python/pyspark/sql/connect/data_frame.py:
##########
@@ -0,0 +1,241 @@
+#
+# 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.
+#
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Sequence,
+    Tuple,
+    Union,
+    cast,
+    TYPE_CHECKING,
+)
+
+import pyspark.sql.connect.plan as plan
+from pyspark.sql.connect.column import (
+    ColumnOrString,
+    ColumnRef,
+    Expression,
+    ExpressionOrString,
+    LiteralExpression,
+)
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+ColumnOrName = Union[ColumnRef, str]
+
+
+class GroupingFrame(object):
+
+    MeasuresType = Union[Sequence[Tuple[ExpressionOrString, str]], Dict[str, str]]
+    OptMeasuresType = Optional[MeasuresType]
+
+    def __init__(self, df: "DataFrame", *grouping_cols: Union[ColumnRef, str]) -> None:
+        self._df = df
+        self._grouping_cols = [x if isinstance(x, ColumnRef) else df[x] for x in grouping_cols]
+
+    def agg(self, exprs: MeasuresType = None) -> "DataFrame":
+
+        # Normalize the dictionary into a list of tuples.
+        if isinstance(exprs, Dict):
+            measures = list(exprs.items())
+        elif isinstance(exprs, List):
+            measures = exprs
+        else:
+            measures = []
+
+        res = DataFrame.withPlan(
+            plan.Aggregate(
+                child=self._df._plan,
+                grouping_cols=self._grouping_cols,
+                measures=measures,
+            ),
+            session=self._df._session,
+        )
+        return res
+
+    def _map_cols_to_dict(self, fun: str, cols: List[Union[ColumnRef, str]]) -> Dict[str, str]:
+        return {x if isinstance(x, str) else cast(ColumnRef, x).name(): fun for x in cols}
+
+    def min(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("min", list(cols))
+        return self.agg(expr)
+
+    def max(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("max", list(cols))
+        return self.agg(expr)
+
+    def sum(self, *cols: Union[ColumnRef, str]) -> "DataFrame":
+        expr = self._map_cols_to_dict("sum", list(cols))
+        return self.agg(expr)
+
+    def count(self) -> "DataFrame":
+        return self.agg([(LiteralExpression(1), "count")])
+
+
+class DataFrame(object):
+    """Every DataFrame object essentially is a Relation that is refined using the
+    member functions. Calling a method on a dataframe will essentially return a copy
+    of the DataFrame with the changes applied.
+    """
+
+    def __init__(self, data: List[Any] = None, schema: List[str] = None):
+        """Creates a new data frame"""
+        self._schema = schema
+        self._plan: Optional[plan.LogicalPlan] = None
+        self._cache: Dict[str, Any] = {}
+        self._session: "RemoteSparkSession" = None
+
+    @classmethod
+    def withPlan(cls, plan: plan.LogicalPlan, session=None) -> "DataFrame":
+        """Main initialization method used to construct a new data frame with a child plan."""
+        new_frame = DataFrame()
+        new_frame._plan = plan
+        new_frame._session = session
+        return new_frame
+
+    def select(self, *cols: ColumnRef) -> "DataFrame":
+        return DataFrame.withPlan(plan.Project(self._plan, *cols), session=self._session)
+
+    def agg(self, exprs: Dict[str, str]) -> "DataFrame":
+        return self.groupBy().agg(exprs)
+
+    def alias(self, alias):
+        return DataFrame.withPlan(plan.Project(self._plan).withAlias(alias), session=self._session)
+
+    def approxQuantile(self, col, probabilities, relativeError):
+        ...
+
+    def colRegex(self, regex) -> "DataFrame":
+        # TODO needs analysis to pick the right column
+        ...
+
+    @property
+    def columns(self) -> List[str]:
+        """Returns the list of columns of the current data frame."""
+        if self._plan is None:
+            return []
+        if "columns" not in self._cache and self._plan is not None:
+            pdd = self.limit(0).collect()
+            # Translate to standard pytho array
+            self._cache["columns"] = pdd.columns.values
+        return self._cache["columns"]
+
+    def count(self):
+        """Returns the number of rows in the data frame"""
+        return self.agg([(LiteralExpression(1), "count")]).collect().iloc[0, 0]
+
+    def crossJoin(self, other):
+        ...
+
+    def coalesce(self, num_partitions: int) -> "DataFrame":
+        # TODO needs repartition operator for substrait
+        ...
+
+    def describe(self, cols):
+        # TODO needs analyze to filter out the right columns
+        ...
+
+    def distinct(self) -> "DataFrame":
+        """Returns all distinct rows."""
+        all_cols = self.columns()
+        gf = self.groupBy(*all_cols)
+        return gf.agg()
+
+    def drop(self, *cols: ColumnOrString):
+        # TODO Needs analyze to know which columns to drop
+        all_cols = self.columns()
+        dropped = set([c.name() if isinstance(c, ColumnRef) else self[c].name() for c in cols])
+        filter(lambda x: x in dropped, all_cols)
+
+    def filter(self, condition: Expression) -> "DataFrame":
+        return DataFrame.withPlan(
+            plan.Filter(child=self._plan, filter=condition), session=self._session
+        )
+
+    def first(self):
+        return self.head(1)
+
+    def groupBy(self, *cols: ColumnOrString):
+        return GroupingFrame(self, *cols)
+
+    def head(self, n: int):
+        self.limit(n)
+        return self.collect()
+
+    def join(self, other, on, how=None):
+        return DataFrame.withPlan(
+            plan.Join(left=self._plan, right=other._plan, on=on, how=how),
+            session=self._session,
+        )
+
+    def limit(self, n):
+        return DataFrame.withPlan(plan.Limit(child=self._plan, limit=n), session=self._session)
+
+    def sort(self, *cols: ColumnOrName):
+        """Sort by a specific column"""
+        return DataFrame.withPlan(plan.Sort(self._plan, *cols), session=self._session)
+
+    def show(self, n: int, truncate: Optional[Union[bool, int]], vertical: Optional[bool]):
+        ...
+
+    def union(self, other) -> "DataFrame":
+        return self.unionAll(other)
+
+    def unionAll(self, other: "DataFrame") -> "DataFrame":
+        if other._plan is None:
+            raise ValueError("Argument to Union does not contain a valid plan.")
+        return DataFrame.withPlan(plan.UnionAll(self._plan, other._plan), session=self._session)
+
+    def where(self, condition):
+        return self.filter(condition)
+
+    def _get_alias(self):
+        p = self._plan
+        while p is not None:
+            if isinstance(p, plan.Project) and p.alias:
+                return p.alias
+            p = p._child
+        return None

Review Comment:
   ```suggestion
   ```



##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.data_frame import DataFrame
+from pyspark.sql.connect.plan import Read
+
+
+class DataFrameReader:
+    def __init__(self, client):
+        self._client = client
+
+    def table(self, tableName: str) -> "DataFrame":
+        df = DataFrame.withPlan(Read(tableName), self._client)
+        return df

Review Comment:
   Can we fail a JIRA to complete the API parity?



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977633416


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.service
+
+import scala.collection.JavaConverters._
+
+import com.google.protobuf.ByteString
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{Request, Response}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.connect.command.SparkConnectCommandPlanner
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.adaptive.{
+  AdaptiveSparkPlanExec,
+  AdaptiveSparkPlanHelper,
+  QueryStageExec
+}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.ArrowUtils
+
+@Experimental
+@Since("3.3.1")
+class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) extends Logging {
+
+  def handle(v: Request): Unit = {
+    // Preconditions.checkState(v.userContext.nonEmpty, "User Context must be present")

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r977642207


##########
python/pyspark/sql/connect/README.md:
##########
@@ -0,0 +1,34 @@
+
+# [EXPERIMENTAL] Spark Connect
+
+**Spark Connect is a strictly experimental feature and under heavy development.
+All APIs should be considered volatile and should not be used in production.**
+
+This module contains the implementation of Spark Connect which is a logical plan
+facade for the implementation in Spark. Spark Connect is directly integrated into the build
+of Spark. To enable it, you only need to activate the driver plugin for Spark Connect.
+
+
+
+
+## Build
+
+1. Build Spark as usual per the documentation.
+2. Build and package the Spark Connect package
+   ```commandline
+   ./build/mvn package

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978407794


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Unstable
+class SparkConnectPlugin extends SparkPlugin {
+
+  /**
+   * Return the plugin's driver-side component.
+   *
+   * @return
+   *   The driver-side component, or null if one is not needed.

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978406868


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _

Review Comment:
   done -> private



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978315881


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable

Review Comment:
   ```
   @Since("3.4.0")
   ```



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978321325


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Unstable
+class SparkConnectPlugin extends SparkPlugin {
+
+  /**
+   * Return the plugin's driver-side component.
+   *
+   * @return
+   *   The driver-side component, or null if one is not needed.

Review Comment:
   nit. Can we put this in a single line?
   ```
   @return The driver-side component, or null if one is not needed.
   ```



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978358109


##########
connect/src/main/scala/org/apache/spark/sql/connect/command/SparkConnectCommandPlanner.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.command
+
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.{Lists, Maps}
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
+import org.apache.spark.sql.types.StringType
+
+@Unstable
+@Since("3.4.0")
+class SparkConnectCommandPlanner(session: SparkSession, command: proto.Command) {
+
+  lazy val pythonVersion =
+    sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))
+
+  def process(): Unit = {
+    command.getCommandTypeCase match {
+      case proto.Command.CommandTypeCase.CREATE_FUNCTION =>
+        handleCreateScalarFunction(command.getCreateFunction)
+      case _ => throw new UnsupportedOperationException(s"${command} not supported.")

Review Comment:
   Done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978419296


##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.execution.ExtendedMode
+
+/**
+ * The SparkConnectService implementation.
+ *
+ * This class implements the service stub from the generated code of GRPC.
+ *
+ * @param debug
+ *   delegates debug behavior to the handlers.
+ */
+@Unstable
+@Since("3.4.0")
+class SparkConnectService(debug: Boolean)
+    extends SparkConnectServiceGrpc.SparkConnectServiceImplBase
+    with Logging {
+
+  /**
+   * This is the main entry method for Spark Connect and all calls to execute a plan.
+   *
+   * The plan execution is delegated to the [[SparkConnectStreamHandler]]. All error handling
+   * should be directly implemented in the deferred implementation. But this method catches
+   * generic errors.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def executePlan(request: Request, responseObserver: StreamObserver[Response]): Unit = {
+    try {
+      new SparkConnectStreamHandler(responseObserver).handle(request)
+    } catch {
+      case e: Throwable =>
+        log.error("Error executing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+
+  /**
+   * Analyze a plan provide metadata and debugging information.
+   *
+   * This method is called to generate the explain plan for a SparkConnect plan. In its simplest
+   * implementation, the plan that is generated by the [[SparkConnectPlanner]] is used to build a
+   * [[Dataset]] and derive the explain string from the query execution details.
+   *
+   * Errors during planning are returned via the [[StreamObserver]] interface.
+   *
+   * @param request
+   * @param responseObserver
+   */
+  override def analyzePlan(
+      request: Request,
+      responseObserver: StreamObserver[AnalyzeResponse]): Unit = {
+    try {
+      val session =
+        SparkConnectService.getOrCreateIsolatedSession(request.getUserContext.getUserId).session
+
+      val logicalPlan = request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.ROOT =>
+          new SparkConnectPlanner(request.getPlan.getRoot, session).transform()
+        case _ =>
+          responseObserver.onError(
+            new UnsupportedOperationException(
+              s"${request.getPlan.getOpTypeCase} not supported for analysis."))
+          return
+      }
+      val ds = Dataset.ofRows(session, logicalPlan)
+      val explainString = ds.queryExecution.explainString(ExtendedMode)
+
+      val resp = proto.AnalyzeResponse
+        .newBuilder()
+        .setExplainString(explainString)
+        .setClientId(request.getClientId)
+
+      resp.addAllColumnTypes(ds.schema.fields.map(_.dataType.sql).toSeq.asJava)
+      resp.addAllColumnNames(ds.schema.fields.map(_.name).toSeq.asJava)
+      responseObserver.onNext(resp.build())
+      responseObserver.onCompleted()
+    } catch {
+      case e: Throwable =>
+        log.error("Error analyzing plan.", e)
+        responseObserver.onError(
+          Status.UNKNOWN.withCause(e).withDescription(e.getLocalizedMessage).asRuntimeException())
+    }
+  }
+}
+
+/**
+ * Trivial object used for referring to SparkSessions in the SessionCache.
+ *
+ * @param userId
+ * @param session
+ */
+@Unstable
+case class SessionHolder(userId: String, session: SparkSession)
+
+/**
+ * Satic instance of the SparkConnectService.
+ *
+ * Used to start the overall SparkConnect service and provides global state to manage the
+ * different SparkSession from different users connecting to the cluster.
+ */
+@Unstable
+object SparkConnectService {
+
+  // Type alias for the SessionCacheKey. Right now this is a String but allows us to switch to a
+  // different or complex type easily.
+  type SessionCacheKey = String;
+
+  var server: Server = _
+
+  private val userSessionMapping =
+    cacheBuilder(100, 3600).build[SessionCacheKey, SessionHolder]()
+
+  // Simple builder for creating the cache of Sessions.
+  private def cacheBuilder(cacheSize: Int, timeoutSeconds: Int): CacheBuilder[Object, Object] = {
+    var cacheBuilder = CacheBuilder.newBuilder().ticker(Ticker.systemTicker())
+    if (cacheSize >= 0) {
+      cacheBuilder = cacheBuilder.maximumSize(cacheSize)
+    }
+    if (timeoutSeconds >= 0) {
+      cacheBuilder.expireAfterAccess(timeoutSeconds, TimeUnit.SECONDS)
+    }
+    cacheBuilder
+  }
+
+  /**
+   * Based on the `key` find or create a new SparkSession.
+   */
+  def getOrCreateIsolatedSession(key: SessionCacheKey): SessionHolder = {
+    userSessionMapping.get(
+      key,
+      () => {
+        SessionHolder(key, newIsolatedSession())
+      })
+  }
+
+  private def newIsolatedSession(): SparkSession = {
+    SparkSession.active.newSession()
+  }
+
+  /**
+   * Starts the GRPC Serivce.
+   *
+   * TODO(SPARK-40536) Make port number configurable.
+   */
+  def startGRPCService(): Unit = {
+    val debugMode = SparkEnv.get.conf.getBoolean("spark.connect.grpc.debug.enabled", true)
+    val port = 15002
+    val sb = NettyServerBuilder
+      .forPort(port)
+      .addService(new SparkConnectService(debugMode))
+
+    // If debug mode is configured, load the ProtoReflection service so that tools like
+    // grpcurl can introspect the API for debugging.
+    if (debugMode) {
+      sb.addService(ProtoReflectionService.newInstance())
+    }
+    server = sb.build
+    server.start()
+  }
+
+  // Starts the service
+  def start(): Unit = {
+    startGRPCService()
+  }
+
+  def stop(): Unit = {
+    if (server != null) {
+      server.shutdownNow()
+    }
+  }
+}
+
+/**
+ * This is the main entry point for Spark Connect.
+ *
+ * To decouple the build of Spark Connect and it's dependencies from the core of Spark, we
+ * implement it as a Driver Plugin. To enable Spark Connect, simply make sure that the appropriate
+ * JAR is available in the CLASSPATH and the driver plugin is configured to load this class.
+ */
+@Unstable
+class SparkConnectPlugin extends SparkPlugin {

Review Comment:
   done.



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


[GitHub] [spark] grundprinzip commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978419138


##########
connect/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar
+}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.types.{
+  BinaryType,
+  ByteType,
+  DateType,
+  DoubleType,
+  FloatType,
+  IntegerType,
+  ShortType,
+  TimestampType
+}

Review Comment:
   Ok, I looked around and understand better now. Adjusted accordingly.



##########
connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.service
+
+import java.util
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.google.common.base.Ticker
+import com.google.common.cache.CacheBuilder
+import io.grpc.{Server, Status}
+import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder
+import io.grpc.protobuf.services.ProtoReflectionService
+import io.grpc.stub.StreamObserver
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.annotation.{Since, Unstable}
+import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin}
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{
+  AnalyzeResponse,
+  Request,
+  Response,
+  SparkConnectServiceGrpc
+}

Review Comment:
   done.



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


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #37710: [SPARK-40448][CONNECT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #37710:
URL: https://github.com/apache/spark/pull/37710#discussion_r978300187


##########
connect/src/main/scala/org/apache/spark/sql/sparkconnect/planner/SparkConnectPlanner.scala:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.sparkconnect.planner
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.connect.proto
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{expressions, plans}
+import org.apache.spark.sql.catalyst.analysis.{
+  UnresolvedAlias,
+  UnresolvedAttribute,
+  UnresolvedFunction,
+  UnresolvedRelation,
+  UnresolvedStar

Review Comment:
   To @HyukjinKwon , although `scalafmt` is recommended for automation, we don't use this style, do we?



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


[GitHub] [spark] grundprinzip commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1239282793

   @Yikun the Docker build works fine again, but I'm still facing the issue with the Doc build that causes my build to fail. It happens in a plain SBT build `sbt clean package -Phive` 
   
   https://github.com/grundprinzip/spark/runs/8226589025?check_suite_focus=true#step:21:6379
   
   Any ideas?


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


[GitHub] [spark] grundprinzip commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1240667644

   Interestingly the build says:
   
   ```
   [info] Main Scala API documentation successful.
   [success] Total time: 159 s (02:39), completed Sep 7, 2022 11:38:25 AM
   Moving back into docs dir.
   Removing old docs
   ```
   
   From what @HyukjinKwon told me is that the unidoc build can have false positives due to the way we generate the Javadoc from the Scala code, and according to the Ruby build script for the documentation, the `system` command finishes with exit code 0 otherwise it would have thrown before.


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


[GitHub] [spark] amaliujia commented on pull request #37710: [DRAFT] Spark Connect build as Driver Plugin with Shaded Dependencies

Posted by GitBox <gi...@apache.org>.
amaliujia commented on PR #37710:
URL: https://github.com/apache/spark/pull/37710#issuecomment-1245771985

   My high level concern about this init PR is that it introduces a lot of code which is beyond what is required for a minimal valued product, and those code are not well tested.
   
   IMO, a MVP is just one that demonstrates the client-server architecture, client can do a single select from a table, and server side execute the plan and return the data. This PR adds a lot more on the API surface, like join, aggregate, etc., which is beyond the MVP requirement. And because those surface are not well tested, I would expect many improvements on those surface to happen.
   
   The direct consequence is we will need to file commits afterwards to update the dumped code. Many will fall into cleanup/bug fix/missing tests, etc. But with a MVP, this could be minimized. The following could be an example that I was thinking of:
   
   With this init PR, we could have some commits in the future:
   `[SPARK-xxxxx][CONNECT] fix outer join implementation due to xxx`
   `[SPARK-xxxxx][CONNECT] add test for non-equality join condition`
   `[SPARK-xxxxx][CONNECT] fix issues for nested join due to xxx`
   `[SPARK-xxxxx][CONNECT] fix issues when join condition contains a function`
   
   With MVP which does not have join on the surface at the beginning, the commits we bring in could be
   `[SPARK-xxxxx][CONNECT] init inner equality join support with tests. Throw exception for non-inner joins`
   `[SPARK-xxxxx][CONNECT][FOLLOWUP] improve inner equality join and more tests`
   `[SPARK-xxxxx][CONNECT][FOLLOWUP] bug fix for inner equality join on xxx`
   `[SPARK-xxxxx][CONNECT] extend join types to outer joins, with tests for situations a, b, c, d...`
   `[SPARK-xxxxx][CONNECT][FOLLOWUP] more tests for outer joins types on uncovered situations f, k, h..`
   
   
   For maintaining the quality of the commit history purpose, the MVP path might be better. 
   
   
   The longer term impact is for the first release we want for the connect project. When we plan to release the first version of it, generally speaking we won't want to release some API that are not well tested or even functional incorrect.  Eventually we either make sure each API is well implemented and tested, or we choose to block those API from being used, which already lead to that we need a version that is implemented good first, then incrementally append high quality changes. Then why don't we target to achieve it at very beginning?
   
   
   My proposal is really to only keep minimal required code that demonstrates the client-server architecture and client can issue a simple operation.
   
   
   This is not blocking this PR and I am open to this concern. As long as Spark community is ok for the big dump, it's good to go.  Just want to make sure some perspectives are covered and we can make good decisions on valid perspectives.
   
   
   
   
   
   
   
   
   
   
      
   
   


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