You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "zhenlineo (via GitHub)" <gi...@apache.org> on 2023/05/23 21:37:27 UTC

[GitHub] [spark] zhenlineo opened a new pull request, #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

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

   ### What changes were proposed in this pull request?
   Expand the client compatibility check to include all sql APIs.
   
   ### Why are the changes needed?
   Enhance the API compatibility coverage
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. This PR modified a few classes to make them package protected to avoid expose unwanted APIs to public.
   
   
   ### How was this patch tested?
   Existing tests.


-- 
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 #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1245132184


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -38,7 +38,7 @@ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode
  *
  * @since 3.5.0
  */
-abstract class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable {
+class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable {

Review Comment:
   due to `class org.apache.spark.sql.KeyValueGroupedDataset was concrete; is declared abstract in client version`



-- 
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] hvanhovell closed pull request #41284: [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell closed pull request #41284: [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list
URL: https://github.com/apache/spark/pull/41284


-- 
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] hvanhovell commented on a diff in pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1244564694


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -122,65 +125,62 @@ object CheckConnectJvmClientCompatibility {
   private def checkMiMaCompatibilityWithSqlModule(
       clientJar: File,
       sqlJar: File): List[Problem] = {
-    val includedRules = Seq(
-      IncludeByName("org.apache.spark.sql.catalog.Catalog.*"),
-      IncludeByName("org.apache.spark.sql.catalog.CatalogMetadata.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Column.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Database.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Function.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Table.*"),
-      IncludeByName("org.apache.spark.sql.Column.*"),
-      IncludeByName("org.apache.spark.sql.ColumnName.*"),
-      IncludeByName("org.apache.spark.sql.DataFrame.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameReader.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameNaFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameStatFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriter.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriterV2.*"),
-      IncludeByName("org.apache.spark.sql.Dataset.*"),
-      IncludeByName("org.apache.spark.sql.functions.*"),
-      IncludeByName("org.apache.spark.sql.KeyValueGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.RelationalGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.SparkSession.*"),
-      IncludeByName("org.apache.spark.sql.RuntimeConfig.*"),
-      IncludeByName("org.apache.spark.sql.TypedColumn.*"),
-      IncludeByName("org.apache.spark.sql.SQLImplicits.*"),
-      IncludeByName("org.apache.spark.sql.DatasetHolder.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamReader.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamWriter.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQuery.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.active"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.get"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.awaitAnyTermination"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.resetTerminated"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryStatus.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryProgress.*"))
+    val includedRules = Seq(IncludeByName("org.apache.spark.sql.*"))
     val excludeRules = Seq(
       // Filter unsupported rules:
       // Note when muting errors for a method, checks on all overloading methods are also muted.
 
-      // Skip all shaded dependencies and proto files in the client.
-      ProblemFilters.exclude[Problem]("org.sparkproject.*"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.connect.proto.*"),
+      // Skip unsupported packages
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.api.*"), // Java, Python, R

Review Comment:
    We need to tackle this at some point.



-- 
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] zhenlineo commented on pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "zhenlineo (via GitHub)" <gi...@apache.org>.
zhenlineo commented on PR #41284:
URL: https://github.com/apache/spark/pull/41284#issuecomment-1608390500

   cc @LuciferYang Could you have a look at this PR? Now the scala client contains enough methods that we can change to a deny list check.


-- 
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] zhenlineo commented on pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "zhenlineo (via GitHub)" <gi...@apache.org>.
zhenlineo commented on PR #41284:
URL: https://github.com/apache/spark/pull/41284#issuecomment-1560166746

   It would be really nice to have https://github.com/apache/spark/pull/40925 to filter package private methods. Looking at MiMa issues. It seems they have been trying to solve the same issue. e.g. https://github.com/lightbend/mima/issues/53 But when using the MiMa check in our tests, we still have to skip the private[sql] classes manually. 


-- 
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] hvanhovell commented on pull request #41284: [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on PR #41284:
URL: https://github.com/apache/spark/pull/41284#issuecomment-1611561044

   Merging.


-- 
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] zhenlineo commented on a diff in pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "zhenlineo (via GitHub)" <gi...@apache.org>.
zhenlineo commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1203078812


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala:
##########
@@ -48,7 +48,7 @@ import org.apache.spark.util.{ThreadUtils, Utils}
  *   An unique identifier of the session which the artifact manager belongs to.
  * @param channel
  */
-class ArtifactManager(
+private[sql] class ArtifactManager(

Review Comment:
   @vicennial Should the artifact and UDF related classes default to be public or package protected?



-- 
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] hvanhovell commented on a diff in pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1244564494


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -122,65 +125,62 @@ object CheckConnectJvmClientCompatibility {
   private def checkMiMaCompatibilityWithSqlModule(
       clientJar: File,
       sqlJar: File): List[Problem] = {
-    val includedRules = Seq(
-      IncludeByName("org.apache.spark.sql.catalog.Catalog.*"),
-      IncludeByName("org.apache.spark.sql.catalog.CatalogMetadata.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Column.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Database.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Function.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Table.*"),
-      IncludeByName("org.apache.spark.sql.Column.*"),
-      IncludeByName("org.apache.spark.sql.ColumnName.*"),
-      IncludeByName("org.apache.spark.sql.DataFrame.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameReader.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameNaFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameStatFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriter.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriterV2.*"),
-      IncludeByName("org.apache.spark.sql.Dataset.*"),
-      IncludeByName("org.apache.spark.sql.functions.*"),
-      IncludeByName("org.apache.spark.sql.KeyValueGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.RelationalGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.SparkSession.*"),
-      IncludeByName("org.apache.spark.sql.RuntimeConfig.*"),
-      IncludeByName("org.apache.spark.sql.TypedColumn.*"),
-      IncludeByName("org.apache.spark.sql.SQLImplicits.*"),
-      IncludeByName("org.apache.spark.sql.DatasetHolder.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamReader.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamWriter.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQuery.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.active"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.get"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.awaitAnyTermination"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.resetTerminated"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryStatus.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryProgress.*"))
+    val includedRules = Seq(IncludeByName("org.apache.spark.sql.*"))
     val excludeRules = Seq(
       // Filter unsupported rules:
       // Note when muting errors for a method, checks on all overloading methods are also muted.
 
-      // Skip all shaded dependencies and proto files in the client.
-      ProblemFilters.exclude[Problem]("org.sparkproject.*"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.connect.proto.*"),
+      // Skip unsupported packages
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.api.*"), // Java, Python, R
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.catalyst.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.columnar.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.connector.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.execution.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.expressions.*"),

Review Comment:
   This is actually public API. What are we still missing 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] zhenlineo commented on a diff in pull request #41284: [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list

Posted by "zhenlineo (via GitHub)" <gi...@apache.org>.
zhenlineo commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1245360835


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -122,65 +125,62 @@ object CheckConnectJvmClientCompatibility {
   private def checkMiMaCompatibilityWithSqlModule(
       clientJar: File,
       sqlJar: File): List[Problem] = {
-    val includedRules = Seq(
-      IncludeByName("org.apache.spark.sql.catalog.Catalog.*"),
-      IncludeByName("org.apache.spark.sql.catalog.CatalogMetadata.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Column.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Database.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Function.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Table.*"),
-      IncludeByName("org.apache.spark.sql.Column.*"),
-      IncludeByName("org.apache.spark.sql.ColumnName.*"),
-      IncludeByName("org.apache.spark.sql.DataFrame.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameReader.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameNaFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameStatFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriter.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriterV2.*"),
-      IncludeByName("org.apache.spark.sql.Dataset.*"),
-      IncludeByName("org.apache.spark.sql.functions.*"),
-      IncludeByName("org.apache.spark.sql.KeyValueGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.RelationalGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.SparkSession.*"),
-      IncludeByName("org.apache.spark.sql.RuntimeConfig.*"),
-      IncludeByName("org.apache.spark.sql.TypedColumn.*"),
-      IncludeByName("org.apache.spark.sql.SQLImplicits.*"),
-      IncludeByName("org.apache.spark.sql.DatasetHolder.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamReader.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamWriter.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQuery.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.active"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.get"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.awaitAnyTermination"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.resetTerminated"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryStatus.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryProgress.*"))
+    val includedRules = Seq(IncludeByName("org.apache.spark.sql.*"))
     val excludeRules = Seq(
       // Filter unsupported rules:
       // Note when muting errors for a method, checks on all overloading methods are also muted.
 
-      // Skip all shaded dependencies and proto files in the client.
-      ProblemFilters.exclude[Problem]("org.sparkproject.*"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.connect.proto.*"),
+      // Skip unsupported packages
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.api.*"), // Java, Python, R
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.catalyst.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.columnar.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.connector.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.execution.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.expressions.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.internal.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.jdbc.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.sources.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.streaming.ui.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.test.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.util.*"),
+
+      // Skip private[sql] constructors
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.*.this"),
+
+      // Skip unsupported classes
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ExperimentalMethods"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$*"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SparkSessionExtensions"),
+      ProblemFilters.exclude[MissingClassProblem](
+        "org.apache.spark.sql.SparkSessionExtensionsProvider"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDTFRegistration"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDFRegistration"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDFRegistration$"),
 
       // DataFrame Reader & Writer
-      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.json"), // deprecated
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.json"), // rdd
 
       // DataFrameNaFunctions
-      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameNaFunctions.this"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameNaFunctions.fillValue"),
 
       // DataFrameStatFunctions
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.bloomFilter"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.this"),
 
       // Dataset
+      ProblemFilters.exclude[MissingClassProblem](
+        "org.apache.spark.sql.Dataset$" // private[sql]
+      ),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.ofRows"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.DATASET_ID_TAG"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.COL_POS_KEY"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.DATASET_ID_KEY"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.curId"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.observe"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation$"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener$"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.queryExecution"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.encoder"),

Review Comment:
   yes. we changed the type from Encoder to AgnosticEncoder



-- 
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] hvanhovell commented on a diff in pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1244565197


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -122,65 +125,62 @@ object CheckConnectJvmClientCompatibility {
   private def checkMiMaCompatibilityWithSqlModule(
       clientJar: File,
       sqlJar: File): List[Problem] = {
-    val includedRules = Seq(
-      IncludeByName("org.apache.spark.sql.catalog.Catalog.*"),
-      IncludeByName("org.apache.spark.sql.catalog.CatalogMetadata.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Column.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Database.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Function.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Table.*"),
-      IncludeByName("org.apache.spark.sql.Column.*"),
-      IncludeByName("org.apache.spark.sql.ColumnName.*"),
-      IncludeByName("org.apache.spark.sql.DataFrame.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameReader.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameNaFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameStatFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriter.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriterV2.*"),
-      IncludeByName("org.apache.spark.sql.Dataset.*"),
-      IncludeByName("org.apache.spark.sql.functions.*"),
-      IncludeByName("org.apache.spark.sql.KeyValueGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.RelationalGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.SparkSession.*"),
-      IncludeByName("org.apache.spark.sql.RuntimeConfig.*"),
-      IncludeByName("org.apache.spark.sql.TypedColumn.*"),
-      IncludeByName("org.apache.spark.sql.SQLImplicits.*"),
-      IncludeByName("org.apache.spark.sql.DatasetHolder.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamReader.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamWriter.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQuery.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.active"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.get"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.awaitAnyTermination"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.resetTerminated"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryStatus.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryProgress.*"))
+    val includedRules = Seq(IncludeByName("org.apache.spark.sql.*"))
     val excludeRules = Seq(
       // Filter unsupported rules:
       // Note when muting errors for a method, checks on all overloading methods are also muted.
 
-      // Skip all shaded dependencies and proto files in the client.
-      ProblemFilters.exclude[Problem]("org.sparkproject.*"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.connect.proto.*"),
+      // Skip unsupported packages
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.api.*"), // Java, Python, R
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.catalyst.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.columnar.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.connector.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.execution.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.expressions.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.internal.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.jdbc.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.sources.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.streaming.ui.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.test.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.util.*"),
+
+      // Skip private[sql] constructors
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.*.this"),
+
+      // Skip unsupported classes
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ExperimentalMethods"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$*"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SparkSessionExtensions"),
+      ProblemFilters.exclude[MissingClassProblem](
+        "org.apache.spark.sql.SparkSessionExtensionsProvider"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDTFRegistration"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDFRegistration"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDFRegistration$"),
 
       // DataFrame Reader & Writer
-      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.json"), // deprecated
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.json"), // rdd
 
       // DataFrameNaFunctions
-      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameNaFunctions.this"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameNaFunctions.fillValue"),
 
       // DataFrameStatFunctions
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.bloomFilter"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.this"),
 
       // Dataset
+      ProblemFilters.exclude[MissingClassProblem](
+        "org.apache.spark.sql.Dataset$" // private[sql]
+      ),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.ofRows"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.DATASET_ID_TAG"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.COL_POS_KEY"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.DATASET_ID_KEY"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.curId"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.observe"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation$"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener"),
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener$"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.queryExecution"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.encoder"),

Review Comment:
   Is this a type mismatch?



-- 
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] hvanhovell commented on a diff in pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1244563139


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -38,7 +38,7 @@ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode
  *
  * @since 3.5.0
  */
-abstract class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable {
+class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable {

Review Comment:
   Why is MiMa complaining about 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] zhenlineo commented on a diff in pull request #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "zhenlineo (via GitHub)" <gi...@apache.org>.
zhenlineo commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1203077171


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala:
##########
@@ -26,7 +26,7 @@ import org.apache.spark.util.Utils
 /**
  * Helper class for conversions between [[DataType]] and [[proto.DataType]].
  */
-object DataTypeProtoConverter {
+private[sql] object DataTypeProtoConverter {

Review Comment:
   @grundprinzip Should the converters in common marked as public? Also see `LiteralValueProtoConverter` and `StorageLevelProtoConverter`
   
   Can we modify them to be package protected?



-- 
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 #41284: [SPARK-43757][Connect] Change client compatibility from allow list to deny list

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #41284:
URL: https://github.com/apache/spark/pull/41284#discussion_r1245142029


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -122,65 +125,62 @@ object CheckConnectJvmClientCompatibility {
   private def checkMiMaCompatibilityWithSqlModule(
       clientJar: File,
       sqlJar: File): List[Problem] = {
-    val includedRules = Seq(
-      IncludeByName("org.apache.spark.sql.catalog.Catalog.*"),
-      IncludeByName("org.apache.spark.sql.catalog.CatalogMetadata.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Column.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Database.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Function.*"),
-      IncludeByName("org.apache.spark.sql.catalog.Table.*"),
-      IncludeByName("org.apache.spark.sql.Column.*"),
-      IncludeByName("org.apache.spark.sql.ColumnName.*"),
-      IncludeByName("org.apache.spark.sql.DataFrame.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameReader.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameNaFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameStatFunctions.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriter.*"),
-      IncludeByName("org.apache.spark.sql.DataFrameWriterV2.*"),
-      IncludeByName("org.apache.spark.sql.Dataset.*"),
-      IncludeByName("org.apache.spark.sql.functions.*"),
-      IncludeByName("org.apache.spark.sql.KeyValueGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.RelationalGroupedDataset.*"),
-      IncludeByName("org.apache.spark.sql.SparkSession.*"),
-      IncludeByName("org.apache.spark.sql.RuntimeConfig.*"),
-      IncludeByName("org.apache.spark.sql.TypedColumn.*"),
-      IncludeByName("org.apache.spark.sql.SQLImplicits.*"),
-      IncludeByName("org.apache.spark.sql.DatasetHolder.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamReader.*"),
-      IncludeByName("org.apache.spark.sql.streaming.DataStreamWriter.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQuery.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.active"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.get"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.awaitAnyTermination"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryManager.resetTerminated"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryStatus.*"),
-      IncludeByName("org.apache.spark.sql.streaming.StreamingQueryProgress.*"))
+    val includedRules = Seq(IncludeByName("org.apache.spark.sql.*"))
     val excludeRules = Seq(
       // Filter unsupported rules:
       // Note when muting errors for a method, checks on all overloading methods are also muted.
 
-      // Skip all shaded dependencies and proto files in the client.
-      ProblemFilters.exclude[Problem]("org.sparkproject.*"),
-      ProblemFilters.exclude[Problem]("org.apache.spark.connect.proto.*"),
+      // Skip unsupported packages
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.api.*"), // Java, Python, R
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.catalyst.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.columnar.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.connector.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.execution.*"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.expressions.*"),

Review Comment:
   like `o.a.s.s.expressions.ReduceAggregator` and `o.a.s.s.expressions.MutableAggregationBuffer`, they do not need to exist in the connect-client module
   
   



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