[HUDI-3567] Refactor HoodieCommonUtils to make code more reasonable (#4982)
This commit is contained in:
@@ -24,9 +24,9 @@ import org.apache.hudi.HoodieBaseRelation.createBaseFileReader
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.SQLContext
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, SQLContext}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression}
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.sources.{BaseRelation, Filter}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
@@ -70,7 +70,8 @@ class BaseFileOnlyViewRelation(sqlContext: SQLContext,
|
||||
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns)
|
||||
|
||||
val filterExpressions = convertToExpressions(filters)
|
||||
val (partitionFilters, dataFilters) = filterExpressions.partition(isPartitionPredicate)
|
||||
val (partitionFilters, dataFilters) = HoodieCatalystExpressionUtils.splitPartitionAndDataPredicates(
|
||||
sparkSession, filterExpressions, partitionColumns)
|
||||
|
||||
val filePartitions = getPartitions(partitionFilters, dataFilters)
|
||||
|
||||
@@ -137,17 +138,4 @@ class BaseFileOnlyViewRelation(sqlContext: SQLContext,
|
||||
|
||||
catalystExpressions.filter(_.isDefined).map(_.get).toArray
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether given expression only references only references partition columns
|
||||
* (and involves no sub-query)
|
||||
*/
|
||||
private def isPartitionPredicate(condition: Expression): Boolean = {
|
||||
// Validates that the provided names both resolve to the same entity
|
||||
val resolvedNameEquals = sparkSession.sessionState.analyzer.resolver
|
||||
|
||||
condition.references.forall { r => partitionColumns.exists(resolvedNameEquals(r.name, _)) } &&
|
||||
!SubqueryExpression.hasSubquery(condition)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.hudi
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.withSparkConf
|
||||
|
||||
import scala.collection.JavaConverters.mapAsJavaMapConverter
|
||||
import scala.collection.immutable.Map
|
||||
|
||||
object HoodieCLIUtils {
|
||||
|
||||
def createHoodieClientFromPath(sparkSession: SparkSession,
|
||||
basePath: String,
|
||||
conf: Map[String, String]): SparkRDDWriteClient[_] = {
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(basePath)
|
||||
.setConf(sparkSession.sessionState.newHadoopConf()).build()
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
|
||||
val finalParameters = HoodieWriterUtils.parametersWithWriteDefaults(
|
||||
withSparkConf(sparkSession, Map.empty)(
|
||||
conf + (DataSourceWriteOptions.TABLE_TYPE.key() -> metaClient.getTableType.name()))
|
||||
)
|
||||
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
DataSourceUtils.createHoodieClient(jsc, schemaStr, basePath,
|
||||
metaClient.getTableConfig.getTableName, finalParameters.asJava)
|
||||
}
|
||||
}
|
||||
@@ -1,286 +0,0 @@
|
||||
/*
|
||||
* 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.hudi
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BoundReference, Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation}
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
import org.apache.spark.sql.execution.datasources.SparkParsePartitionUtil
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.withSparkConf
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import scala.collection.JavaConverters.mapAsJavaMapConverter
|
||||
import scala.collection.immutable.Map
|
||||
|
||||
object HoodieCommonUtils extends Logging {
|
||||
|
||||
def createHoodieClientFromPath(sparkSession: SparkSession, basePath: String,
|
||||
conf: Map[String, String]): SparkRDDWriteClient[_] = {
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(basePath)
|
||||
.setConf(sparkSession.sessionState.newHadoopConf()).build()
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
|
||||
val finalParameters = HoodieWriterUtils.parametersWithWriteDefaults(
|
||||
withSparkConf(sparkSession, Map.empty)(
|
||||
conf + (DataSourceWriteOptions.TABLE_TYPE.key() -> metaClient.getTableType.name()))
|
||||
)
|
||||
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
DataSourceUtils.createHoodieClient(jsc, schemaStr, basePath,
|
||||
metaClient.getTableConfig.getTableName, finalParameters.asJava)
|
||||
}
|
||||
|
||||
def getPartitionSchemaFromProperty(metaClient: HoodieTableMetaClient,
|
||||
schemaSpec: Option[StructType]): StructType = {
|
||||
val schema = schemaSpec.getOrElse({
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema)
|
||||
})
|
||||
|
||||
val tableConfig = metaClient.getTableConfig
|
||||
val partitionColumns = tableConfig.getPartitionFields
|
||||
val nameFieldMap = generateFieldMap(schema)
|
||||
|
||||
if (partitionColumns.isPresent) {
|
||||
// Note that key generator class name could be null
|
||||
val keyGeneratorClassName = tableConfig.getKeyGeneratorClassName
|
||||
if (classOf[TimestampBasedKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)
|
||||
|| classOf[TimestampBasedAvroKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)) {
|
||||
val partitionFields = partitionColumns.get().map(column => StructField(column, StringType))
|
||||
StructType(partitionFields)
|
||||
} else {
|
||||
val partitionFields = partitionColumns.get().map(column =>
|
||||
nameFieldMap.getOrElse(column, throw new IllegalArgumentException(s"Cannot find column: '" +
|
||||
s"$column' in the schema[${schema.fields.mkString(",")}]")))
|
||||
StructType(partitionFields)
|
||||
}
|
||||
} else {
|
||||
// If the partition columns have not stored in hoodie.properties(the table that was
|
||||
// created earlier), we trait it as a non-partitioned table.
|
||||
logWarning("No partition columns available from hoodie.properties." +
|
||||
" Partition pruning will not work")
|
||||
new StructType()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method unravels [[StructType]] into a [[Map]] of pairs of dot-path notation with corresponding
|
||||
* [[StructField]] object for every field of the provided [[StructType]], recursively.
|
||||
*
|
||||
* For example, following struct
|
||||
* <pre>
|
||||
* StructType(
|
||||
* StructField("a",
|
||||
* StructType(
|
||||
* StructField("b", StringType),
|
||||
* StructField("c", IntType)
|
||||
* )
|
||||
* )
|
||||
* )
|
||||
* </pre>
|
||||
*
|
||||
* will be converted into following mapping:
|
||||
*
|
||||
* <pre>
|
||||
* "a.b" -> StructField("b", StringType),
|
||||
* "a.c" -> StructField("c", IntType),
|
||||
* </pre>
|
||||
*/
|
||||
def generateFieldMap(structType: StructType): Map[String, StructField] = {
|
||||
def traverse(structField: Either[StructField, StructType]): Map[String, StructField] = {
|
||||
structField match {
|
||||
case Right(struct) => struct.fields.flatMap(f => traverse(Left(f))).toMap
|
||||
case Left(field) => field.dataType match {
|
||||
case struct: StructType => traverse(Right(struct)).map {
|
||||
case (key, structField) => (s"${field.name}.$key", structField)
|
||||
}
|
||||
case _ => Map(field.name -> field)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
traverse(Right(structType))
|
||||
}
|
||||
|
||||
/**
|
||||
* Prune the partition by the filter.This implementation is fork from
|
||||
* org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex#prunePartitions.
|
||||
*
|
||||
* @param partitionPaths All the partition paths.
|
||||
* @param predicates The filter condition.
|
||||
* @return The Pruned partition paths.
|
||||
*/
|
||||
def prunePartition(partitionSchema: StructType,
|
||||
partitionPaths: Seq[PartitionPath],
|
||||
predicates: Seq[Expression]): Seq[PartitionPath] = {
|
||||
val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
|
||||
val partitionPruningPredicates = predicates.filter {
|
||||
_.references.map(_.name).toSet.subsetOf(partitionColumnNames)
|
||||
}
|
||||
if (partitionPruningPredicates.nonEmpty) {
|
||||
val predicate = partitionPruningPredicates.reduce(expressions.And)
|
||||
prunePartition(partitionSchema, partitionPaths, predicate)
|
||||
} else {
|
||||
partitionPaths
|
||||
}
|
||||
}
|
||||
|
||||
def prunePartition(partitionSchema: StructType,
|
||||
partitionPaths: Seq[PartitionPath],
|
||||
predicate: Expression): Seq[PartitionPath] = {
|
||||
val boundPredicate = InterpretedPredicate(predicate.transform {
|
||||
case a: AttributeReference =>
|
||||
val index = partitionSchema.indexWhere(a.name == _.name)
|
||||
BoundReference(index, partitionSchema(index).dataType, nullable = true)
|
||||
})
|
||||
|
||||
val prunedPartitionPaths = partitionPaths.filter {
|
||||
partitionPath => boundPredicate.eval(InternalRow.fromSeq(partitionPath.values))
|
||||
}
|
||||
|
||||
logInfo(s"Total partition size is: ${partitionPaths.size}," +
|
||||
s" after partition prune size is: ${prunedPartitionPaths.size}")
|
||||
prunedPartitionPaths
|
||||
}
|
||||
|
||||
def parsePartitionColumnValues(sparkParsePartitionUtil: SparkParsePartitionUtil,
|
||||
configProperties: TypedProperties,
|
||||
basePath: String,
|
||||
partitionSchema: StructType,
|
||||
partitionColumns: Array[String],
|
||||
partitionPath: String): Array[Object] = {
|
||||
if (partitionColumns.length == 0) {
|
||||
// This is a non-partitioned table
|
||||
Array.empty
|
||||
} else {
|
||||
val partitionFragments = partitionPath.split("/")
|
||||
|
||||
if (partitionFragments.length != partitionColumns.length &&
|
||||
partitionColumns.length == 1) {
|
||||
// If the partition column size is not equal to the partition fragment size
|
||||
// and the partition column size is 1, we map the whole partition path
|
||||
// to the partition column which can benefit from the partition prune.
|
||||
val prefix = s"${partitionColumns.head}="
|
||||
val partitionValue = if (partitionPath.startsWith(prefix)) {
|
||||
// support hive style partition path
|
||||
partitionPath.substring(prefix.length)
|
||||
} else {
|
||||
partitionPath
|
||||
}
|
||||
Array(UTF8String.fromString(partitionValue))
|
||||
} else if (partitionFragments.length != partitionColumns.length &&
|
||||
partitionColumns.length > 1) {
|
||||
// If the partition column size is not equal to the partition fragments size
|
||||
// and the partition column size > 1, we do not know how to map the partition
|
||||
// fragments to the partition columns. So we trait it as a Non-Partitioned Table
|
||||
// for the query which do not benefit from the partition prune.
|
||||
logWarning(s"Cannot do the partition prune for table $basePath." +
|
||||
s"The partitionFragments size (${partitionFragments.mkString(",")})" +
|
||||
s" is not equal to the partition columns size(${partitionColumns.mkString(",")})")
|
||||
Array.empty
|
||||
} else {
|
||||
// If partitionSeqs.length == partitionSchema.fields.length
|
||||
// Append partition name to the partition value if the
|
||||
// HIVE_STYLE_PARTITIONING is disable.
|
||||
// e.g. convert "/xx/xx/2021/02" to "/xx/xx/year=2021/month=02"
|
||||
val partitionWithName =
|
||||
partitionFragments.zip(partitionColumns).map {
|
||||
case (partition, columnName) =>
|
||||
if (partition.indexOf("=") == -1) {
|
||||
s"${columnName}=$partition"
|
||||
} else {
|
||||
partition
|
||||
}
|
||||
}.mkString("/")
|
||||
|
||||
val pathWithPartitionName = new Path(basePath, partitionWithName)
|
||||
val partitionValues = parsePartitionPath(sparkParsePartitionUtil, configProperties, basePath,
|
||||
pathWithPartitionName, partitionSchema)
|
||||
|
||||
partitionValues.map(_.asInstanceOf[Object]).toArray
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def parsePartitionPath(sparkParsePartitionUtil: SparkParsePartitionUtil,
|
||||
configProperties: TypedProperties,
|
||||
basePath: String,
|
||||
partitionPath: Path,
|
||||
partitionSchema: StructType): Seq[Any] = {
|
||||
val timeZoneId = configProperties.getString(DateTimeUtils.TIMEZONE_OPTION, SQLConf.get.sessionLocalTimeZone)
|
||||
val partitionDataTypes = partitionSchema.map(f => f.name -> f.dataType).toMap
|
||||
|
||||
sparkParsePartitionUtil.parsePartition(
|
||||
partitionPath,
|
||||
typeInference = false,
|
||||
Set(new Path(basePath)),
|
||||
partitionDataTypes,
|
||||
DateTimeUtils.getTimeZone(timeZoneId)
|
||||
)
|
||||
.toSeq(partitionSchema)
|
||||
}
|
||||
|
||||
def getConfigProperties(spark: SparkSession, options: Map[String, String]): TypedProperties = {
|
||||
val sqlConf: SQLConf = spark.sessionState.conf
|
||||
val properties = new TypedProperties()
|
||||
|
||||
// To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users
|
||||
// would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing.
|
||||
properties.setProperty(HoodieMetadataConfig.ENABLE.key(),
|
||||
sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
|
||||
properties.putAll(options.asJava)
|
||||
properties
|
||||
}
|
||||
|
||||
def resolveFilterExpr(spark: SparkSession, exprString: String, tableSchema: StructType): Expression = {
|
||||
val expr = spark.sessionState.sqlParser.parseExpression(exprString)
|
||||
resolveFilterExpr(spark, expr, tableSchema)
|
||||
}
|
||||
|
||||
def resolveFilterExpr(spark: SparkSession, expr: Expression, tableSchema: StructType): Expression = {
|
||||
val schemaFields = tableSchema.fields
|
||||
val resolvedExpr = spark.sessionState.analyzer.ResolveReferences(
|
||||
Filter(expr, LocalRelation(schemaFields.head, schemaFields.drop(1): _*))
|
||||
)
|
||||
.asInstanceOf[Filter].condition
|
||||
|
||||
checkForUnresolvedRefs(resolvedExpr)
|
||||
}
|
||||
|
||||
private def checkForUnresolvedRefs(resolvedExpr: Expression): Expression =
|
||||
resolvedExpr match {
|
||||
case UnresolvedAttribute(_) => throw new IllegalStateException("unresolved attribute")
|
||||
case _ => resolvedExpr.mapChildren(e => checkForUnresolvedRefs(e))
|
||||
}
|
||||
}
|
||||
@@ -19,6 +19,7 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
|
||||
import org.apache.hudi.HoodieFileIndex.getConfigProperties
|
||||
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.util.StringUtils
|
||||
@@ -36,10 +37,11 @@ import org.apache.spark.sql.types.{StringType, StructType}
|
||||
import org.apache.spark.sql.{AnalysisException, Column, SparkSession}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import java.text.SimpleDateFormat
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
import scala.util.{Failure, Success, Try}
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
import java.text.SimpleDateFormat
|
||||
|
||||
/**
|
||||
* A file index which support partition prune for hoodie snapshot and read-optimized query.
|
||||
@@ -73,7 +75,7 @@ case class HoodieFileIndex(spark: SparkSession,
|
||||
spark = spark,
|
||||
metaClient = metaClient,
|
||||
schemaSpec = schemaSpec,
|
||||
configProperties = HoodieCommonUtils.getConfigProperties(spark, options),
|
||||
configProperties = getConfigProperties(spark, options),
|
||||
queryPaths = Seq(HoodieFileIndex.getQueryPath(options)),
|
||||
specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key).map(HoodieSqlCommonUtils.formatQueryInstant),
|
||||
fileStatusCache = fileStatusCache
|
||||
@@ -147,8 +149,7 @@ case class HoodieFileIndex(spark: SparkSession,
|
||||
Seq(PartitionDirectory(InternalRow.empty, candidateFiles))
|
||||
} else {
|
||||
// Prune the partition path by the partition filters
|
||||
val prunedPartitions = HoodieCommonUtils.prunePartition(partitionSchema,
|
||||
cachedAllInputFileSlices.keySet.asScala.toSeq, convertedPartitionFilters)
|
||||
val prunedPartitions = prunePartition(cachedAllInputFileSlices.keySet.asScala.toSeq, convertedPartitionFilters)
|
||||
var totalFileSize = 0
|
||||
var candidateFileSize = 0
|
||||
|
||||
|
||||
@@ -18,18 +18,24 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath
|
||||
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL}
|
||||
import org.apache.hudi.SparkHoodieTableFileIndex.{deduceQueryType, toJavaOption}
|
||||
import org.apache.hudi.SparkHoodieTableFileIndex.{deduceQueryType, generateFieldMap, toJavaOption}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.model.{FileSlice, HoodieTableQueryType}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BoundReference, Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.language.implicitConversions
|
||||
@@ -78,8 +84,32 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
|
||||
/**
|
||||
* Get the partition schema from the hoodie.properties.
|
||||
*/
|
||||
private lazy val _partitionSchemaFromProperties: StructType =
|
||||
HoodieCommonUtils.getPartitionSchemaFromProperty(metaClient, Some(schema))
|
||||
private lazy val _partitionSchemaFromProperties: StructType = {
|
||||
val tableConfig = metaClient.getTableConfig
|
||||
val partitionColumns = tableConfig.getPartitionFields
|
||||
val nameFieldMap = generateFieldMap(schema)
|
||||
|
||||
if (partitionColumns.isPresent) {
|
||||
// Note that key generator class name could be null
|
||||
val keyGeneratorClassName = tableConfig.getKeyGeneratorClassName
|
||||
if (classOf[TimestampBasedKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)
|
||||
|| classOf[TimestampBasedAvroKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)) {
|
||||
val partitionFields = partitionColumns.get().map(column => StructField(column, StringType))
|
||||
StructType(partitionFields)
|
||||
} else {
|
||||
val partitionFields = partitionColumns.get().map(column =>
|
||||
nameFieldMap.getOrElse(column, throw new IllegalArgumentException(s"Cannot find column: '" +
|
||||
s"$column' in the schema[${schema.fields.mkString(",")}]")))
|
||||
StructType(partitionFields)
|
||||
}
|
||||
} else {
|
||||
// If the partition columns have not stored in hoodie.properties(the table that was
|
||||
// created earlier), we trait it as a non-partitioned table.
|
||||
logWarning("No partition columns available from hoodie.properties." +
|
||||
" Partition pruning will not work")
|
||||
new StructType()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the data schema of the table.
|
||||
@@ -110,16 +140,121 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
|
||||
*/
|
||||
def listFileSlices(partitionFilters: Seq[Expression]): Map[String, Seq[FileSlice]] = {
|
||||
// Prune the partition path by the partition filters
|
||||
val prunedPartitions = HoodieCommonUtils.prunePartition(partitionSchema,
|
||||
cachedAllInputFileSlices.asScala.keys.toSeq, partitionFilters)
|
||||
val prunedPartitions = prunePartition(cachedAllInputFileSlices.asScala.keys.toSeq, partitionFilters)
|
||||
prunedPartitions.map(partition => {
|
||||
(partition.path, cachedAllInputFileSlices.get(partition).asScala)
|
||||
}).toMap
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all the cached partition paths pruned by the filter.
|
||||
*
|
||||
* @param predicates The filter condition
|
||||
* @return The pruned partition paths
|
||||
*/
|
||||
def getPartitionPaths(predicates: Seq[Expression]): Seq[PartitionPath] = {
|
||||
prunePartition(cachedAllInputFileSlices.keySet().asScala.toSeq, predicates)
|
||||
}
|
||||
|
||||
/**
|
||||
* Prune the partition by the filter.This implementation is fork from
|
||||
* org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex#prunePartitions.
|
||||
*
|
||||
* @param partitionPaths All the partition paths.
|
||||
* @param predicates The filter condition.
|
||||
* @return The pruned partition paths.
|
||||
*/
|
||||
protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
|
||||
val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
|
||||
val partitionPruningPredicates = predicates.filter {
|
||||
_.references.map(_.name).toSet.subsetOf(partitionColumnNames)
|
||||
}
|
||||
if (partitionPruningPredicates.nonEmpty) {
|
||||
val predicate = partitionPruningPredicates.reduce(expressions.And)
|
||||
|
||||
val boundPredicate = InterpretedPredicate(predicate.transform {
|
||||
case a: AttributeReference =>
|
||||
val index = partitionSchema.indexWhere(a.name == _.name)
|
||||
BoundReference(index, partitionSchema(index).dataType, nullable = true)
|
||||
})
|
||||
|
||||
val prunedPartitionPaths = partitionPaths.filter {
|
||||
partitionPath => boundPredicate.eval(InternalRow.fromSeq(partitionPath.values))
|
||||
}
|
||||
|
||||
logInfo(s"Total partition size is: ${partitionPaths.size}," +
|
||||
s" after partition prune size is: ${prunedPartitionPaths.size}")
|
||||
prunedPartitionPaths
|
||||
} else {
|
||||
partitionPaths
|
||||
}
|
||||
}
|
||||
|
||||
protected def parsePartitionColumnValues(partitionColumns: Array[String], partitionPath: String): Array[Object] = {
|
||||
HoodieCommonUtils.parsePartitionColumnValues(sparkParsePartitionUtil, configProperties,
|
||||
basePath, partitionSchema, partitionColumns, partitionPath)
|
||||
if (partitionColumns.length == 0) {
|
||||
// This is a non-partitioned table
|
||||
Array.empty
|
||||
} else {
|
||||
val partitionFragments = partitionPath.split("/")
|
||||
|
||||
if (partitionFragments.length != partitionColumns.length &&
|
||||
partitionColumns.length == 1) {
|
||||
// If the partition column size is not equal to the partition fragment size
|
||||
// and the partition column size is 1, we map the whole partition path
|
||||
// to the partition column which can benefit from the partition prune.
|
||||
val prefix = s"${partitionColumns.head}="
|
||||
val partitionValue = if (partitionPath.startsWith(prefix)) {
|
||||
// support hive style partition path
|
||||
partitionPath.substring(prefix.length)
|
||||
} else {
|
||||
partitionPath
|
||||
}
|
||||
Array(UTF8String.fromString(partitionValue))
|
||||
} else if (partitionFragments.length != partitionColumns.length &&
|
||||
partitionColumns.length > 1) {
|
||||
// If the partition column size is not equal to the partition fragments size
|
||||
// and the partition column size > 1, we do not know how to map the partition
|
||||
// fragments to the partition columns. So we trait it as a Non-Partitioned Table
|
||||
// for the query which do not benefit from the partition prune.
|
||||
logWarning(s"Cannot do the partition prune for table $basePath." +
|
||||
s"The partitionFragments size (${partitionFragments.mkString(",")})" +
|
||||
s" is not equal to the partition columns size(${partitionColumns.mkString(",")})")
|
||||
Array.empty
|
||||
} else {
|
||||
// If partitionSeqs.length == partitionSchema.fields.length
|
||||
// Append partition name to the partition value if the
|
||||
// HIVE_STYLE_PARTITIONING is disable.
|
||||
// e.g. convert "/xx/xx/2021/02" to "/xx/xx/year=2021/month=02"
|
||||
val partitionWithName =
|
||||
partitionFragments.zip(partitionColumns).map {
|
||||
case (partition, columnName) =>
|
||||
if (partition.indexOf("=") == -1) {
|
||||
s"${columnName}=$partition"
|
||||
} else {
|
||||
partition
|
||||
}
|
||||
}.mkString("/")
|
||||
|
||||
val pathWithPartitionName = new Path(basePath, partitionWithName)
|
||||
val partitionValues = parsePartitionPath(pathWithPartitionName, partitionSchema)
|
||||
|
||||
partitionValues.map(_.asInstanceOf[Object]).toArray
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def parsePartitionPath(partitionPath: Path, partitionSchema: StructType): Seq[Any] = {
|
||||
val timeZoneId = configProperties.getString(DateTimeUtils.TIMEZONE_OPTION, SQLConf.get.sessionLocalTimeZone)
|
||||
val partitionDataTypes = partitionSchema.map(f => f.name -> f.dataType).toMap
|
||||
|
||||
sparkParsePartitionUtil.parsePartition(
|
||||
partitionPath,
|
||||
typeInference = false,
|
||||
Set(new Path(basePath)),
|
||||
partitionDataTypes,
|
||||
DateTimeUtils.getTimeZone(timeZoneId)
|
||||
)
|
||||
.toSeq(partitionSchema)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -132,6 +267,45 @@ object SparkHoodieTableFileIndex {
|
||||
org.apache.hudi.common.util.Option.empty()
|
||||
}
|
||||
|
||||
/**
|
||||
* This method unravels [[StructType]] into a [[Map]] of pairs of dot-path notation with corresponding
|
||||
* [[StructField]] object for every field of the provided [[StructType]], recursively.
|
||||
*
|
||||
* For example, following struct
|
||||
* <pre>
|
||||
* StructType(
|
||||
* StructField("a",
|
||||
* StructType(
|
||||
* StructField("b", StringType),
|
||||
* StructField("c", IntType)
|
||||
* )
|
||||
* )
|
||||
* )
|
||||
* </pre>
|
||||
*
|
||||
* will be converted into following mapping:
|
||||
*
|
||||
* <pre>
|
||||
* "a.b" -> StructField("b", StringType),
|
||||
* "a.c" -> StructField("c", IntType),
|
||||
* </pre>
|
||||
*/
|
||||
private def generateFieldMap(structType: StructType) : Map[String, StructField] = {
|
||||
def traverse(structField: Either[StructField, StructType]) : Map[String, StructField] = {
|
||||
structField match {
|
||||
case Right(struct) => struct.fields.flatMap(f => traverse(Left(f))).toMap
|
||||
case Left(field) => field.dataType match {
|
||||
case struct: StructType => traverse(Right(struct)).map {
|
||||
case (key, structField) => (s"${field.name}.$key", structField)
|
||||
}
|
||||
case _ => Map(field.name -> field)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
traverse(Right(structType))
|
||||
}
|
||||
|
||||
private def deduceQueryType(configProperties: TypedProperties): HoodieTableQueryType = {
|
||||
configProperties.asScala(QUERY_TYPE.key()) match {
|
||||
case QUERY_TYPE_SNAPSHOT_OPT_VAL => HoodieTableQueryType.SNAPSHOT
|
||||
|
||||
@@ -0,0 +1,88 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
object HoodieCatalystExpressionUtils {
|
||||
|
||||
/**
|
||||
* Resolve filter expression from string expr with given table schema, for example:
|
||||
* <pre>
|
||||
* ts > 1000 and ts <= 1500
|
||||
* </pre>
|
||||
* will be resolved as
|
||||
* <pre>
|
||||
* And(GreaterThan(ts#590L > 1000), LessThanOrEqual(ts#590L <= 1500))
|
||||
* </pre>
|
||||
*
|
||||
* @param spark The spark session
|
||||
* @param exprString String to be resolved
|
||||
* @param tableSchema The table schema
|
||||
* @return Resolved filter expression
|
||||
*/
|
||||
def resolveFilterExpr(spark: SparkSession, exprString: String, tableSchema: StructType): Expression = {
|
||||
val expr = spark.sessionState.sqlParser.parseExpression(exprString)
|
||||
resolveFilterExpr(spark, expr, tableSchema)
|
||||
}
|
||||
|
||||
def resolveFilterExpr(spark: SparkSession, expr: Expression, tableSchema: StructType): Expression = {
|
||||
val schemaFields = tableSchema.fields
|
||||
val resolvedExpr = spark.sessionState.analyzer.ResolveReferences(
|
||||
Filter(expr,
|
||||
LocalRelation(schemaFields.head, schemaFields.drop(1): _*))
|
||||
)
|
||||
.asInstanceOf[Filter].condition
|
||||
|
||||
checkForUnresolvedRefs(resolvedExpr)
|
||||
}
|
||||
|
||||
private def checkForUnresolvedRefs(resolvedExpr: Expression): Expression =
|
||||
resolvedExpr match {
|
||||
case UnresolvedAttribute(_) => throw new IllegalStateException("unresolved attribute")
|
||||
case _ => resolvedExpr.mapChildren(e => checkForUnresolvedRefs(e))
|
||||
}
|
||||
|
||||
/**
|
||||
* Split the given predicates into two sequence predicates:
|
||||
* - predicates that references partition columns only(and involves no sub-query);
|
||||
* - other predicates.
|
||||
*
|
||||
* @param sparkSession The spark session
|
||||
* @param predicates The predicates to be split
|
||||
* @param partitionColumns The partition columns
|
||||
* @return (partitionFilters, dataFilters)
|
||||
*/
|
||||
def splitPartitionAndDataPredicates(sparkSession: SparkSession,
|
||||
predicates: Array[Expression],
|
||||
partitionColumns: Array[String]): (Array[Expression], Array[Expression]) = {
|
||||
// Validates that the provided names both resolve to the same entity
|
||||
val resolvedNameEquals = sparkSession.sessionState.analyzer.resolver
|
||||
|
||||
predicates.partition(expr => {
|
||||
// Checks whether given expression only references partition columns(and involves no sub-query)
|
||||
expr.references.forall(r => partitionColumns.exists(resolvedNameEquals(r.name, _))) &&
|
||||
!SubqueryExpression.hasSubquery(expr)
|
||||
})
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user