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/31 18:20:10 UTC

[GitHub] [spark] huaxingao opened a new pull request, #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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

   
   
   ### What changes were proposed in this pull request?
   Add more info in the v2 table error message to make it more meaningful.
   
   
   ### Why are the changes needed?
   When V2 catalog is not configured, Spark fails to access/create a table using the V2 API and silently falls back to attempting to do the same operation using the V1 Api. This happens frequently among the users. We want to have a better error message so that users can fix the configuration/usage issue by themselves.
   
   
   ### Does this PR introduce _any_ user-facing change?
   no
   
   
   ### 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] amaliujia commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -844,10 +846,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
       s"Cannot delete from table ${table.name} where ${filters.mkString("[", ", ", "]")}")
   }
 
-  def deleteOnlySupportedWithV2TablesError(): Throwable = {

Review Comment:
   Why did? Is it not longer used after this factoring? 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -844,10 +846,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
       s"Cannot delete from table ${table.name} where ${filters.mkString("[", ", ", "]")}")
   }
 
-  def deleteOnlySupportedWithV2TablesError(): Throwable = {

Review Comment:
   Why delete? Is it not longer used after this factoring? 



-- 
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] huaxingao commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "TABLE_OPERATION" : {
+        "message" : [
+          "Table '<catalog>'.'<nameSpace>'.'<tableName>' does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by \"spark.sql.catalog.spark_catalog\"."

Review Comment:
   Updated. Thanks!



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -541,8 +541,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
     new AnalysisException("ADD COLUMN with v1 tables cannot specify NOT NULL.")
   }
 
-  def operationOnlySupportedWithV2TableError(operation: String): Throwable = {
-    new AnalysisException(s"$operation is only supported with v2 tables.")
+  def operationOnlySupportedWithV2TableError(
+      catalog: String,
+      nameSpace: String,
+      tableName: String,
+      operation: String): Throwable = {
+    new AnalysisException(
+      errorClass = "UNSUPPORTED_FEATURE",
+      errorSubClass = "TABLE_OPERATION",
+      messageParameters = Array(catalog, nameSpace, tableName, operation))

Review Comment:
   Fixed



##########
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala:
##########
@@ -50,28 +50,44 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
     case AddColumns(ResolvedV1TableIdentifier(ident), cols) =>
       cols.foreach { c =>
-        assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand")
+        if (c.name.length > 1) {
+          throw QueryCompilationErrors.operationOnlySupportedWithV2TableError(
+            ident.catalog.getOrElse(""),
+            ident.database.getOrElse(""),

Review Comment:
   Fixed. 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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r960284889


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {

Review Comment:
   let's make it short as it's a sub-error-class. How about `TABLE_OPERATION`



-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r963257429


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -565,6 +565,11 @@
           "<property> is a reserved table property, <msg>."
         ]
       },
+      "TABLE_OPERATION" : {
+        "message" : [
+          "Table <tableName> does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by \"spark.sql.catalog.spark_catalog\"."

Review Comment:
   ```suggestion
             "Table <tableName> does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by \"spark.sql.catalog.<catalogName>\"."
   ```



-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r961379227


##########
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala:
##########
@@ -50,28 +50,44 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
     case AddColumns(ResolvedV1TableIdentifier(ident), cols) =>
       cols.foreach { c =>
-        assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand")
+        if (c.name.length > 1) {
+          throw QueryCompilationErrors.operationOnlySupportedWithV2TableError(
+            ident.catalog.getOrElse(""),
+            ident.database.getOrElse(""),

Review Comment:
   `ident` must be qualified, we can do `ident.catalog.get`



-- 
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] MaxGekk commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {
+        "message" : [
+          "Table `spark_catalog`.`default`.`t1` does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by 'spark.sql.catalog.spark_catalog'."

Review Comment:
   spark.sql.catalog.spark_catalog should be quoted by "", see other configs in error-classes.json.



-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r961379227


##########
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala:
##########
@@ -50,28 +50,44 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
     case AddColumns(ResolvedV1TableIdentifier(ident), cols) =>
       cols.foreach { c =>
-        assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand")
+        if (c.name.length > 1) {
+          throw QueryCompilationErrors.operationOnlySupportedWithV2TableError(
+            ident.catalog.getOrElse(""),
+            ident.database.getOrElse(""),

Review Comment:
   `ident` must be qualified, we can do `ident.catalog.get` and `ident.database.get`



-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r961378514


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -541,8 +541,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
     new AnalysisException("ADD COLUMN with v1 tables cannot specify NOT NULL.")
   }
 
-  def operationOnlySupportedWithV2TableError(operation: String): Throwable = {
-    new AnalysisException(s"$operation is only supported with v2 tables.")
+  def operationOnlySupportedWithV2TableError(
+      catalog: String,
+      nameSpace: String,
+      tableName: String,
+      operation: String): Throwable = {
+    new AnalysisException(
+      errorClass = "UNSUPPORTED_FEATURE",
+      errorSubClass = "TABLE_OPERATION",
+      messageParameters = Array(catalog, nameSpace, tableName, operation))

Review Comment:
   for passing table name, we should pass `toSQLId(nameParts)`



-- 
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] MaxGekk commented on pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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

   +1, LGTM. Merging to master.
   Thank you, @huaxingao and @cloud-fan @amaliujia for review.


-- 
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] huaxingao commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {
+        "message" : [
+          "Table `spark_catalog`.`default`.`t1` does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by 'spark.sql.catalog.spark_catalog'."

Review Comment:
   Fixed. 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] huaxingao commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {

Review Comment:
   Sounds good. Changed.



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {
+        "message" : [
+          "Table `spark_catalog`.`default`.`t1` does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by 'spark.sql.catalog.spark_catalog'."

Review Comment:
   Fixed. 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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r960213508


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -542,7 +542,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def operationOnlySupportedWithV2TableError(operation: String): Throwable = {
-    new AnalysisException(s"$operation is only supported with v2 tables.")
+    new AnalysisException(s"$operation is only supported with v2 tables. To use" +
+      s" v2 tables, please config the catalog correctly using spark.sql.catalog" +

Review Comment:
   if catalog is not configured correctly, I think users will see table not found exception?



-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r963258197


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -541,8 +541,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
     new AnalysisException("ADD COLUMN with v1 tables cannot specify NOT NULL.")
   }
 
-  def operationOnlySupportedWithV2TableError(operation: String): Throwable = {
-    new AnalysisException(s"$operation is only supported with v2 tables.")
+  def operationOnlySupportedWithV2TableError(
+      nameParts: Seq[String],
+      operation: String): Throwable = {
+    new AnalysisException(
+      errorClass = "UNSUPPORTED_FEATURE",
+      errorSubClass = "TABLE_OPERATION",
+      messageParameters = Array(toSQLId(nameParts), toSQLStmt(operation)))

Review Comment:
   we may pass `ADD COLUMN with qualified column` as the operation. we shouldn't call `toSQLStmt` 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] huaxingao commented on pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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

   Thanks a lot! @MaxGekk @cloud-fan @amaliujia 


-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r961377593


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "TABLE_OPERATION" : {
+        "message" : [
+          "Table '<catalog>'.'<nameSpace>'.'<tableName>' does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by \"spark.sql.catalog.spark_catalog\"."

Review Comment:
   ```suggestion
             "Table <tableName> does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by \"spark.sql.catalog.<catalog>\"."
   ```



-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r960285081


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {
+        "message" : [
+          "Table `spark_catalog`.`default`.`t1` does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by 'spark.sql.catalog.spark_catalog'."

Review Comment:
   let's pass the real table name



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

To unsubscribe, e-mail: reviews-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] huaxingao commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -541,8 +541,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
     new AnalysisException("ADD COLUMN with v1 tables cannot specify NOT NULL.")
   }
 
-  def operationOnlySupportedWithV2TableError(operation: String): Throwable = {
-    new AnalysisException(s"$operation is only supported with v2 tables.")
+  def operationOnlySupportedWithV2TableError(
+      nameParts: Seq[String],
+      operation: String): Throwable = {
+    new AnalysisException(
+      errorClass = "UNSUPPORTED_FEATURE",
+      errorSubClass = "TABLE_OPERATION",
+      messageParameters = Array(toSQLId(nameParts), operation))

Review Comment:
   Fixed. 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] MaxGekk closed pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
MaxGekk closed pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful
URL: https://github.com/apache/spark/pull/37746


-- 
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] MaxGekk commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala:
##########
@@ -1257,8 +1257,8 @@ class PlanResolutionSuite extends AnalysisTest {
           val e2 = intercept[AnalysisException] {
             parseAndResolve(sql4)
           }
-          assert(e2.getMessage.contains(
-            "ALTER COLUMN with qualified column is only supported with v2 tables"))
+          assert(e2.getMessage.contains("Table `spark_catalog`.`default`.`v1Table` does not " +

Review Comment:
   Please, use `checkError()` and avoid checking the error message from `error-classes.json` that can be in local language or modified by tech editors independently from your test.



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

To unsubscribe, e-mail: reviews-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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r960285847


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {
+        "message" : [
+          "Table `spark_catalog`.`default`.`t1` does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by 'spark.sql.catalog.spark_catalog'."

Review Comment:
   and also the catalog name



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -520,6 +520,11 @@
           "NATURAL CROSS JOIN."
         ]
       },
+      "OPERATION_ONLY_SUPPORTED_WITH_V2_TABLE" : {
+        "message" : [
+          "Table `spark_catalog`.`default`.`t1` does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by 'spark.sql.catalog.spark_catalog'."

Review Comment:
   and also the real catalog name



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

To unsubscribe, e-mail: reviews-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] huaxingao commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -844,10 +846,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
       s"Cannot delete from table ${table.name} where ${filters.mkString("[", ", ", "]")}")
   }
 
-  def deleteOnlySupportedWithV2TablesError(): Throwable = {

Review Comment:
   We can use `operationOnlySupportedWithV2TableError(operation: String)` at L544



-- 
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] MaxGekk commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -541,8 +541,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
     new AnalysisException("ADD COLUMN with v1 tables cannot specify NOT NULL.")
   }
 
-  def operationOnlySupportedWithV2TableError(operation: String): Throwable = {
-    new AnalysisException(s"$operation is only supported with v2 tables.")
+  def operationOnlySupportedWithV2TableError(
+      nameParts: Seq[String],
+      operation: String): Throwable = {
+    new AnalysisException(
+      errorClass = "UNSUPPORTED_FEATURE",
+      errorSubClass = "TABLE_OPERATION",
+      messageParameters = Array(toSQLId(nameParts), operation))

Review Comment:
   The operation is a part of SQL statement, correct? Please, wrap it by `toSQLStmt()`.



-- 
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] MaxGekk commented on pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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

   @huaxingao Could you fix PlanResolutionSuite and SparkThrowableSuite, please.


-- 
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] huaxingao commented on pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

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

   cc @cloud-fan 


-- 
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] cloud-fan commented on a diff in pull request #37746: [SPARK-40293][SQL] Make the V2 table error message more meaningful

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37746:
URL: https://github.com/apache/spark/pull/37746#discussion_r963257578


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -565,6 +565,11 @@
           "<property> is a reserved table property, <msg>."
         ]
       },
+      "TABLE_OPERATION" : {
+        "message" : [
+          "Table <tableName> does not support <operation>. Please check the current catalog and namespace to make sure the qualified table name is expected, and also check the catalog implementation which is configured by \"spark.sql.catalog.spark_catalog\"."

Review Comment:
   people can configure catalog implementation for every catalog.



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