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 <biyan900...@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") + ) + } +}