[HUDI-2456] support 'show partitions' sql (#3693)
This commit is contained in:
@@ -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) =>
|
||||
|
||||
@@ -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) =>
|
||||
|
||||
@@ -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"
|
||||
|
||||
@@ -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]
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user