This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch branch-0.x
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/branch-0.x by this push:
     new 2b7ebe5afac6 [HUDI-8161] Make spark-sql command 'desc' independent 
from schema evolution config (#17858)
2b7ebe5afac6 is described below

commit 2b7ebe5afac60469d9fa2858a38603e69c4c86e8
Author: Lin Liu <[email protected]>
AuthorDate: Mon Feb 9 20:10:25 2026 -0800

    [HUDI-8161] Make spark-sql command 'desc' independent from schema evolution 
config (#17858)
    
    
    ---------
    
    Co-authored-by: Vova Kolmakov <[email protected]>
    Co-authored-by: Vova Kolmakov <[email protected]>
---
 .../spark/sql/hudi/ddl/TestDescribeTable.scala     | 117 +++++++++++++++++++++
 .../hudi/analysis/HoodieSpark32PlusAnalysis.scala  |   4 +
 2 files changed, 121 insertions(+)

diff --git 
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDescribeTable.scala
 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDescribeTable.scala
new file mode 100644
index 000000000000..0bcfaf4023ab
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestDescribeTable.scala
@@ -0,0 +1,117 @@
+/*
+ * 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.hudi.ddl
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase
+
+import java.util.function.Predicate
+
+class TestDescribeTable extends HoodieSparkSqlTestBase {
+
+  test("Test desc hudi table command") {
+    withTempDir { tmp =>
+      val tbName = "wk_date"
+      val basePath = s"${tmp.getCanonicalPath}/$tbName"
+
+      spark.sql(
+        s"""
+           |create table $tbName (id int, driver string, precomb int, dat 
string)
+           | using hudi
+           | partitioned by(dat)
+           | tblproperties(
+           |   type='cow',
+           |   primaryKey='id',
+           |   preCombineField='precomb'
+           | )
+           | location '$basePath'
+       """.stripMargin)
+
+      // just for scala-2.11 compatibility
+      val locationInFirstColumn: Predicate[Row] = new Predicate[Row] {
+        def test(row: Row): Boolean = row(0).equals("Location")
+      }
+
+      spark.sql("set hoodie.schema.on.read.enable=false")
+      var output: java.util.List[Row] = spark.sql(s"describe extended 
$tbName").collectAsList()
+      assert(output.stream().anyMatch(locationInFirstColumn))
+
+      spark.sql("set hoodie.schema.on.read.enable=true")
+      output = spark.sql(s"desc formatted $tbName").collectAsList()
+      assert(output.stream().anyMatch(locationInFirstColumn))
+
+      output = spark.sql(s"describe table extended $tbName").collectAsList()
+      assert(output.stream().anyMatch(locationInFirstColumn))
+
+      // DESC returns only columns and partitions when run without 'extended' 
or 'formatted' keywords
+      output = spark.sql(s"describe table $tbName").collectAsList()
+      assert(output.stream().noneMatch(locationInFirstColumn))
+
+      output = spark.sql(s"desc table $tbName").collectAsList()
+      assert(output.stream().noneMatch(locationInFirstColumn))
+
+      output = spark.sql(s"desc $tbName").collectAsList()
+      assert(output.stream().noneMatch(locationInFirstColumn))
+    }
+  }
+
+  test("Test desc non-hudi table command") {
+    withTempDir { tmp =>
+      val tbName = "wk_date"
+      val basePath = s"${tmp.getCanonicalPath}/$tbName"
+
+      spark.sql(
+        s"""
+           |create table $tbName (
+           | id int,
+           | driver string,
+           | precomb int,
+           | dat string
+           |)
+           | using parquet
+           | location '$basePath'
+       """.stripMargin)
+
+      // just for scala-2.11 compatibility
+      val locationInFirstColumn: Predicate[Row] = new Predicate[Row] {
+        def test(row: Row): Boolean = row(0).equals("Location")
+      }
+
+      spark.sql("set hoodie.schema.on.read.enable=false")
+      var output: java.util.List[Row] = spark.sql(s"describe extended 
$tbName").collectAsList()
+      assert(output.stream().anyMatch(locationInFirstColumn))
+
+      spark.sql("set hoodie.schema.on.read.enable=true")
+      output = spark.sql(s"desc formatted $tbName").collectAsList()
+      assert(output.stream().anyMatch(locationInFirstColumn))
+
+      output = spark.sql(s"describe table extended $tbName").collectAsList()
+      assert(output.stream().anyMatch(locationInFirstColumn))
+
+      // DESC returns only columns and partitions when run without 'extended' 
or 'formatted' keywords
+      output = spark.sql(s"describe table $tbName").collectAsList()
+      assert(output.stream().noneMatch(locationInFirstColumn))
+
+      output = spark.sql(s"desc table $tbName").collectAsList()
+      assert(output.stream().noneMatch(locationInFirstColumn))
+
+      output = spark.sql(s"desc $tbName").collectAsList()
+      assert(output.stream().noneMatch(locationInFirstColumn))
+    }
+  }
+}
diff --git 
a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala
 
b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala
index ccf991cb6930..ecdfdcdb81cb 100644
--- 
a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala
+++ 
b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import 
org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
 import org.apache.spark.sql.connector.catalog.{Table, V1Table}
 import 
org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.execution.command.DescribeTableCommand
 import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
 import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField
 import org.apache.spark.sql.hudi.ProvidesHoodieConfig
@@ -259,6 +260,9 @@ case class HoodieSpark32PlusPostAnalysisRule(sparkSession: 
SparkSession) extends
           retainData = true
         )
 
+      case DescribeRelation(MatchResolvedTable(_, id, HoodieV1OrV2Table(_)), 
partitionSpec, isExtended, output) =>
+        DescribeTableCommand(id.asTableIdentifier, partitionSpec, isExtended, 
output)
+
       case _ => plan
     }
   }

Reply via email to