You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by le...@apache.org on 2021/10/06 07:47:02 UTC

[hudi] branch master updated: [HUDI-2456] support 'show partitions' sql (#3693)

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

leesf pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new e91e611  [HUDI-2456] support 'show partitions' sql (#3693)
e91e611 is described below

commit e91e611afbee2958322b2e32f41faabc5918f6b4
Author: Yann Byron <bi...@gmail.com>
AuthorDate: Wed Oct 6 15:46:49 2021 +0800

    [HUDI-2456] support 'show partitions' sql (#3693)
---
 .../java/org/apache/hudi/keygen/KeyGenUtils.java   |   8 +-
 .../apache/hudi/keygen/RowKeyGeneratorHelper.java  |  18 +--
 .../hudi/keygen/TimestampBasedKeyGenerator.java    |   4 +-
 .../apache/hudi/keygen/TestSimpleKeyGenerator.java |   8 +-
 .../hudi/common/util/PartitionPathEncodeUtils.java |  19 ++-
 .../org/apache/spark/sql/hudi/HoodieSqlUtils.scala |  17 ++-
 .../spark/sql/hudi/analysis/HoodieAnalysis.scala   |   9 +-
 .../hudi/command/CreateHoodieTableCommand.scala    |  18 +--
 .../command/ShowHoodieTablePartitionsCommand.scala |  76 ++++++++++
 .../apache/spark/sql/hudi/TestHoodieSqlBase.scala  |   2 +-
 .../apache/spark/sql/hudi/TestShowPartitions.scala | 164 +++++++++++++++++++++
 11 files changed, 297 insertions(+), 46 deletions(-)

diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
index 4923d98..8038afe 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
@@ -40,7 +40,7 @@ public class KeyGenUtils {
   protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
   protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
 
-  protected static final String DEFAULT_PARTITION_PATH = "default";
+  protected static final String HUDI_DEFAULT_PARTITION_PATH = PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH;
   public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
 
   /**
@@ -121,8 +121,8 @@ public class KeyGenUtils {
     for (String partitionPathField : partitionPathFields) {
       String fieldVal = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
       if (fieldVal == null || fieldVal.isEmpty()) {
-        partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH
-            : DEFAULT_PARTITION_PATH);
+        partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + HUDI_DEFAULT_PARTITION_PATH
+            : HUDI_DEFAULT_PARTITION_PATH);
       } else {
         if (encodePartitionPath) {
           fieldVal = PartitionPathEncodeUtils.escapePathName(fieldVal);
@@ -147,7 +147,7 @@ public class KeyGenUtils {
       boolean hiveStylePartitioning, boolean encodePartitionPath) {
     String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
     if (partitionPath == null || partitionPath.isEmpty()) {
-      partitionPath = DEFAULT_PARTITION_PATH;
+      partitionPath = HUDI_DEFAULT_PARTITION_PATH;
     }
     if (encodePartitionPath) {
       partitionPath = PartitionPathEncodeUtils.escapePathName(partitionPath);
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java
index 329fdd7..5c6a0e4 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java
@@ -38,7 +38,7 @@ import java.util.stream.IntStream;
 
 import scala.Option;
 
-import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
+import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
 import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR;
 import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
 import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
@@ -104,11 +104,11 @@ public class RowKeyGeneratorHelper {
         Integer fieldPos = fieldPositions.get(0);
         // for partition path, if field is not found, index will be set to -1
         if (fieldPos == -1 || row.isNullAt(fieldPos)) {
-          val = DEFAULT_PARTITION_PATH;
+          val = HUDI_DEFAULT_PARTITION_PATH;
         } else {
           val = row.getAs(field).toString();
           if (val.isEmpty()) {
-            val = DEFAULT_PARTITION_PATH;
+            val = HUDI_DEFAULT_PARTITION_PATH;
           }
         }
         if (hiveStylePartitioning) {
@@ -117,7 +117,7 @@ public class RowKeyGeneratorHelper {
       } else { // nested
         Object nestedVal = getNestedFieldVal(row, partitionPathPositions.get(field));
         if (nestedVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER) || nestedVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
-          val = hiveStylePartitioning ? field + "=" + DEFAULT_PARTITION_PATH : DEFAULT_PARTITION_PATH;
+          val = hiveStylePartitioning ? field + "=" + HUDI_DEFAULT_PARTITION_PATH : HUDI_DEFAULT_PARTITION_PATH;
         } else {
           val = hiveStylePartitioning ? field + "=" + nestedVal.toString() : nestedVal.toString();
         }
@@ -137,11 +137,11 @@ public class RowKeyGeneratorHelper {
         Integer fieldPos = fieldPositions.get(0);
         // for partition path, if field is not found, index will be set to -1
         if (fieldPos == -1 || row.isNullAt(fieldPos)) {
-          val = DEFAULT_PARTITION_PATH;
+          val = HUDI_DEFAULT_PARTITION_PATH;
         } else {
           Object value = row.get(fieldPos, partitionPathDataTypes.get(field).get(0));
           if (value == null || value.toString().isEmpty()) {
-            val = DEFAULT_PARTITION_PATH;
+            val = HUDI_DEFAULT_PARTITION_PATH;
           } else {
             val = value.toString();
           }
@@ -161,11 +161,11 @@ public class RowKeyGeneratorHelper {
                                                   DataType partitionPathDataType) {
     Object val = null;
     if (internalRow.isNullAt(partitionPathPosition)) {
-      return DEFAULT_PARTITION_PATH;
+      return HUDI_DEFAULT_PARTITION_PATH;
     } else {
       Object value = partitionPathDataType == DataTypes.StringType ? internalRow.getString(partitionPathPosition) : internalRow.get(partitionPathPosition, partitionPathDataType);
       if (value == null || value.toString().isEmpty()) {
-        val = DEFAULT_PARTITION_PATH;
+        val = HUDI_DEFAULT_PARTITION_PATH;
       } else {
         val = value;
       }
@@ -197,7 +197,7 @@ public class RowKeyGeneratorHelper {
    */
   public static Object getNestedFieldVal(Row row, List<Integer> positions) {
     if (positions.size() == 1 && positions.get(0) == -1) {
-      return DEFAULT_PARTITION_PATH;
+      return HUDI_DEFAULT_PARTITION_PATH;
     }
     int index = 0;
     int totalCount = positions.size();
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
index 03d1e92..e3a5a33 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
@@ -29,7 +29,7 @@ import org.apache.spark.sql.types.StructType;
 
 import java.io.IOException;
 
-import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
+import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
 import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
 import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
 
@@ -85,7 +85,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
   private String getTimestampBasedPartitionPath(Object partitionPathFieldVal) {
     Object fieldVal = null;
     try {
-      if (partitionPathFieldVal == null || partitionPathFieldVal.toString().contains(DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER)
+      if (partitionPathFieldVal == null || partitionPathFieldVal.toString().contains(HUDI_DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER)
           || partitionPathFieldVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
         fieldVal = timestampBasedAvroKeyGenerator.getDefaultPartitionVal();
       } else {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
index 75dca2e..0fc90c8 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
@@ -34,7 +34,7 @@ import org.junit.jupiter.params.provider.MethodSource;
 
 import java.util.stream.Stream;
 
-import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
+import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
 
 public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
   private TypedProperties getCommonProps() {
@@ -108,9 +108,9 @@ public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
   public void testWrongPartitionPathField() {
     SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps());
     GenericRecord record = getRecord();
-    Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.DEFAULT_PARTITION_PATH);
+    Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH);
     Assertions.assertEquals(keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)),
-        KeyGenUtils.DEFAULT_PARTITION_PATH);
+        KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH);
   }
 
   @Test
@@ -151,7 +151,7 @@ public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
       partitionPathFieldValue = (String) nestedColRecord.get("prop1");
     }
     String expectedPartitionPath = "nested_col.prop1="
-        + (partitionPathFieldValue != null && !partitionPathFieldValue.isEmpty() ? partitionPathFieldValue : DEFAULT_PARTITION_PATH);
+        + (partitionPathFieldValue != null && !partitionPathFieldValue.isEmpty() ? partitionPathFieldValue : HUDI_DEFAULT_PARTITION_PATH);
     HoodieKey key = keyGenerator.getKey(record);
     Assertions.assertEquals("key1", key.getRecordKey());
     Assertions.assertEquals(expectedPartitionPath, key.getPartitionPath());
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/PartitionPathEncodeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/PartitionPathEncodeUtils.java
index a63a529..e489143 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/PartitionPathEncodeUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/PartitionPathEncodeUtils.java
@@ -25,6 +25,8 @@ import java.util.BitSet;
  */
 public class PartitionPathEncodeUtils {
 
+  public static final String DEFAULT_PARTITION_PATH = "default";
+
   static BitSet charToEscape = new BitSet(128);
   static {
     for (char c = 0; c < ' '; c++) {
@@ -64,14 +66,11 @@ public class PartitionPathEncodeUtils {
    * @return An escaped path name.
    */
   public static String escapePathName(String path, String defaultPath) {
-
-    // __HIVE_DEFAULT_NULL__ is the system default value for null and empty string.
-    // TODO: we should allow user to specify default partition or HDFS file location.
     if (path == null || path.length() == 0) {
       if (defaultPath == null) {
-        //previously, when path is empty or null and no default path is specified,
-        // __HIVE_DEFAULT_PARTITION__ was the return value for escapePathName
-        return "__HIVE_DEFAULT_PARTITION__";
+        // previously, when path is empty or null and no default path is specified,
+        // "default" was the return value for escapePathName
+        return DEFAULT_PARTITION_PATH;
       } else {
         return defaultPath;
       }
@@ -111,4 +110,12 @@ public class PartitionPathEncodeUtils {
     }
     return sb.toString();
   }
+
+  public static String escapePartitionValue(String value) {
+    if (value == null || value.isEmpty()) {
+      return DEFAULT_PARTITION_PATH;
+    } else {
+      return escapePathName(value);
+    }
+  }
 }
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala
index c1130d2..318577b 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala
@@ -19,10 +19,14 @@ package org.apache.spark.sql.hudi
 
 import scala.collection.JavaConverters._
 import java.net.URI
-import java.util.{Date, Locale}
+import java.util.{Date, Locale, Properties}
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hudi.SparkAdapterSupport
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.fs.FSUtils
 import org.apache.hudi.common.model.HoodieRecord
 import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
@@ -36,6 +40,7 @@ import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expressi
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, MergeIntoTable, SubqueryAlias}
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.sql.types.{DataType, NullType, StringType, StructField, StructType}
 
 import java.text.SimpleDateFormat
@@ -80,6 +85,16 @@ object HoodieSqlUtils extends SparkAdapterSupport {
       .asInstanceOf[StructType]).map(removeMetaFields)
   }
 
+  def getAllPartitionPaths(spark: SparkSession, table: CatalogTable): Seq[String] = {
+    val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext))
+    val metadataConfig = {
+      val properties = new Properties()
+      properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties).asJava)
+      HoodieMetadataConfig.newBuilder.fromProperties(properties).build()
+    }
+    FSUtils.getAllPartitionPaths(sparkEngine, metadataConfig, HoodieSqlUtils.getTableLocation(table, spark)).asScala
+  }
+
   private def tripAlias(plan: LogicalPlan): LogicalPlan = {
     plan match {
       case SubqueryAlias(_, relation: LogicalPlan) =>
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
index a588eb6..3a6bedf 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
@@ -29,13 +29,13 @@ import org.apache.spark.sql.catalyst.expressions.AttributeReference
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, Literal, NamedExpression}
 import org.apache.spark.sql.catalyst.plans.Inner
-import org.apache.spark.sql.catalyst.plans.logical.{Assignment, CompactionPath, CompactionShowOnPath, CompactionShowOnTable, CompactionTable, DeleteAction, DeleteFromTable, InsertAction, LogicalPlan, MergeIntoTable, Project, UpdateAction, UpdateTable}
+import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableChangeColumnCommand, AlterTableRenameCommand, CreateDataSourceTableCommand, TruncateTableCommand}
+import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation}
 import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
-import org.apache.spark.sql.hudi.command.{AlterHoodieTableAddColumnsCommand, AlterHoodieTableChangeColumnCommand, AlterHoodieTableRenameCommand, CompactionHoodiePathCommand, CompactionHoodieTableCommand, CompactionShowHoodiePathCommand, CompactionShowHoodieTableCommand, CreateHoodieTableAsSelectCommand, CreateHoodieTableCommand, DeleteHoodieTableCommand, InsertIntoHoodieTableCommand, MergeIntoHoodieTableCommand, TruncateHoodieTableCommand, UpdateHoodieTableCommand}
+import org.apache.spark.sql.hudi.command._
 import org.apache.spark.sql.types.StringType
 
 object HoodieAnalysis {
@@ -417,6 +417,9 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic
       case AlterTableChangeColumnCommand(tableName, columnName, newColumn)
         if isHoodieTable(tableName, sparkSession) =>
         AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn)
+      case ShowPartitionsCommand(tableName, specOpt)
+        if isHoodieTable(tableName, sparkSession) =>
+         ShowHoodieTablePartitionsCommand(tableName, specOpt)
       // Rewrite TruncateTableCommand to TruncateHoodieTableCommand
       case TruncateTableCommand(tableName, partitionSpec)
         if isHoodieTable(tableName, sparkSession) =>
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
index 00e8afb..3129863 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
@@ -19,9 +19,6 @@ package org.apache.spark.sql.hudi.command
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
-import org.apache.hudi.client.common.HoodieSparkEngineContext
-import org.apache.hudi.common.config.HoodieMetadataConfig
-import org.apache.hudi.common.fs.FSUtils
 import org.apache.hudi.common.model.HoodieFileFormat
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
 import org.apache.hudi.common.util.ValidationUtils
@@ -29,7 +26,6 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat
 import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
 import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
 import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
-import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.avro.SchemaConverters
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -129,9 +125,9 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
         (addMetaFields(tableSchema.get), options)
       } else if (userSpecifiedSchema.nonEmpty) {
         (addMetaFields(userSpecifiedSchema), options)
-    } else {
+      } else {
         throw new IllegalArgumentException(s"Missing schema for Create Table: $tableName")
-     }
+      }
     } else {
       assert(table.schema.nonEmpty, s"Missing schema for Create Table: $tableName")
       // SPARK-19724: the default location of a managed table should be non-existent or empty.
@@ -319,16 +315,6 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
     }
   }
 
-  private def getAllPartitionPaths(spark: SparkSession, table: CatalogTable): Seq[String] = {
-    val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext))
-    val metadataConfig = {
-      val properties = new Properties()
-      properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties).asJava)
-      HoodieMetadataConfig.newBuilder.fromProperties(properties).build()
-    }
-    FSUtils.getAllPartitionPaths(sparkEngine, metadataConfig, getTableLocation(table, spark)).asScala
-  }
-
   /**
    * This method is used to compatible with the old non-hive-styled partition table.
    * By default we enable the "hoodie.datasource.write.hive_style_partitioning"
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
new file mode 100644
index 0000000..1c1f4b7
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.command
+
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.util.PartitionPathEncodeUtils
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.execution.datasources.PartitioningUtils
+import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Command for show hudi table's partitions.
+ */
+case class ShowHoodieTablePartitionsCommand(
+    tableName: TableIdentifier,
+    specOpt: Option[TablePartitionSpec])
+extends RunnableCommand {
+
+  override val output: Seq[Attribute] = {
+    AttributeReference("partition", StringType, nullable = false)() :: Nil
+  }
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val catalog = sparkSession.sessionState.catalog
+    val resolver = sparkSession.sessionState.conf.resolver
+    val catalogTable = catalog.getTableMetadata(tableName)
+    val tablePath = getTableLocation(catalogTable, sparkSession)
+
+    val hadoopConf = sparkSession.sessionState.newHadoopConf()
+    val metaClient = HoodieTableMetaClient.builder().setBasePath(tablePath)
+      .setConf(hadoopConf).build()
+    val schemaOpt = getTableSqlSchema(metaClient)
+    val partitionColumnNamesOpt = metaClient.getTableConfig.getPartitionFields
+    if (partitionColumnNamesOpt.isPresent && partitionColumnNamesOpt.get.nonEmpty
+        && schemaOpt.isDefined && schemaOpt.nonEmpty) {
+
+      val partitionColumnNames = partitionColumnNamesOpt.get
+      val schema = schemaOpt.get
+      val allPartitionPaths: Seq[String] = getAllPartitionPaths(sparkSession, catalogTable)
+
+      if (specOpt.isEmpty) {
+        allPartitionPaths.map(Row(_))
+      } else {
+        val spec = specOpt.get
+        allPartitionPaths.filter { partitionPath =>
+          val part = PartitioningUtils.parsePathFragment(partitionPath)
+          spec.forall { case (col, value) =>
+            PartitionPathEncodeUtils.escapePartitionValue(value) == part.getOrElse(col, null)
+          }
+        }.map(Row(_))
+      }
+    } else {
+      Seq.empty[Row]
+    }
+  }
+}
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala
index e35b9b7..1f9b1ea 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala
@@ -77,7 +77,7 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
   }
 
   protected def checkAnswer(sql: String)(expects: Seq[Any]*): Unit = {
-    assertResult(expects.map(row => Row(row: _*)).toArray)(spark.sql(sql).collect())
+    assertResult(expects.map(row => Row(row: _*)).toArray.sortBy(_.toString()))(spark.sql(sql).collect().sortBy(_.toString()))
   }
 
   protected def checkException(sql: String)(errorMsg: String): Unit = {
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala
new file mode 100644
index 0000000..05ee61c
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala
@@ -0,0 +1,164 @@
+/*
+ * 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
+
+import org.apache.spark.sql.Row
+
+class TestShowPartitions extends TestHoodieSqlBase {
+
+  test("Test Show Non Partitioned Table's Partitions") {
+    val tableName = generateTableName
+    // Create a non-partitioned table
+    spark.sql(
+      s"""
+         | create table $tableName (
+         |  id int,
+         |  name string,
+         |  price double,
+         |  ts long
+         |) using hudi
+         |options (
+         |  primaryKey = 'id',
+         |  preCombineField = 'ts'
+         )
+       """.stripMargin)
+    // Insert data
+    spark.sql(
+      s"""
+         | insert into $tableName
+         | select 1 as id, 'a1' as name, 10 as price, 1000 as ts
+        """.stripMargin)
+    checkAnswer(s"show partitions $tableName")(Seq.empty: _*)
+  }
+
+  test("Test Show Partitioned Table's Partitions") {
+    val tableName = generateTableName
+    // Create a partitioned table
+    spark.sql(
+      s"""
+         | create table $tableName (
+         |  id int,
+         |  name string,
+         | price double,
+         |  ts long,
+         |  dt string
+         ) using hudi
+         | partitioned by (dt)
+         | options (
+         |   primaryKey = 'id',
+         |   preCombineField = 'ts'
+         | )
+       """.stripMargin)
+    // Empty partitions
+    checkAnswer(s"show partitions $tableName")(Seq.empty: _*)
+
+    // Insert into dynamic partition
+    spark.sql(
+      s"""
+         | insert into $tableName
+         | values (1, 'a1', 10, 1000, '2021-01-01')
+        """.stripMargin)
+    checkAnswer(s"show partitions $tableName")(Seq("dt=2021-01-01"))
+
+    // Insert into static partition
+    spark.sql(
+      s"""
+         | insert into $tableName partition(dt = '2021-01-02')
+         | select 2 as id, 'a2' as name, 10 as price, 1000 as ts
+        """.stripMargin)
+    checkAnswer(s"show partitions $tableName partition(dt='2021-01-02')")(Seq("dt=2021-01-02"))
+
+    // Insert into null partition
+    spark.sql(
+      s"""
+         | insert into $tableName
+         | select 3 as id, 'a3' as name, 10 as price, 1000 as ts, null as dt
+        """.stripMargin)
+    checkAnswer(s"show partitions $tableName")(
+      Seq("dt=2021-01-01"), Seq("dt=2021-01-02"), Seq("dt=default")
+    )
+  }
+
+  test("Test Show Table's Partitions with MultiLevel Partitions") {
+    val tableName = generateTableName
+    // Create a multi-level partitioned table
+    spark.sql(
+      s"""
+         | create table $tableName (
+         |   id int,
+         |   name string,
+         |   price double,
+         |   ts long,
+         |   year string,
+         |   month string,
+         |   day string
+         | ) using hudi
+         | partitioned by (year, month, day)
+         | options (
+         |   primaryKey = 'id',
+         |   preCombineField = 'ts'
+         | )
+       """.stripMargin)
+    // Empty partitions
+    checkAnswer(s"show partitions $tableName")(Seq.empty: _*)
+
+    // Insert into dynamic partition
+    spark.sql(
+      s"""
+         | insert into $tableName
+         | values
+         |   (1, 'a1', 10, 1000, '2021', '01', '01'),
+         |   (2, 'a2', 10, 1000, '2021', '01', '02'),
+         |   (3, 'a3', 10, 1000, '2021', '02', '01'),
+         |   (4, 'a4', 10, 1000, '2021', '02', null),
+         |   (5, 'a5', 10, 1000, '2021', null, '01'),
+         |   (6, 'a6', 10, 1000, null, '01', '02'),
+         |   (7, 'a6', 10, 1000, '2022', null, null),
+         |   (8, 'a6', 10, 1000, null, '01', null),
+         |   (9, 'a6', 10, 1000, null, null, '01')
+        """.stripMargin)
+
+    // check all partitions
+    checkAnswer(s"show partitions $tableName")(
+      Seq("year=2021/month=01/day=01"),
+      Seq("year=2021/month=01/day=02"),
+      Seq("year=2021/month=02/day=01"),
+      Seq("year=2021/month=02/day=default"),
+      Seq("year=2021/month=default/day=01"),
+      Seq("year=default/month=01/day=default"),
+      Seq("year=default/month=01/day=02"),
+      Seq("year=default/month=default/day=01"),
+      Seq("year=2022/month=default/day=default")
+    )
+
+    // check partial partitions
+    checkAnswer(s"show partitions $tableName partition(year='2021', month='01', day='01')")(
+      Seq("year=2021/month=01/day=01")
+    )
+    checkAnswer(s"show partitions $tableName partition(year='2021', month='02')")(
+      Seq("year=2021/month=02/day=default"),
+      Seq("year=2021/month=02/day=01")
+    )
+    checkAnswer(s"show partitions $tableName partition(day=01)")(
+      Seq("year=2021/month=02/day=01"),
+      Seq("year=2021/month=default/day=01"),
+      Seq("year=2021/month=01/day=01"),
+      Seq("year=default/month=default/day=01")
+    )
+  }
+}