Skip to content

Commit

Permalink
[HUDI-3254] Introduce HoodieCatalog to manage tables for Spark Dataso…
Browse files Browse the repository at this point in the history
…urce V2 (apache#4611)
  • Loading branch information
leesf authored Feb 14, 2022
1 parent 5ca4480 commit 0db1e97
Show file tree
Hide file tree
Showing 26 changed files with 1,288 additions and 81 deletions.
7 changes: 6 additions & 1 deletion .github/workflows/bot.yml
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,16 @@ jobs:
include:
- scala: "scala-2.11"
spark: "spark2"
skipModules: ""
- scala: "scala-2.11"
spark: "spark2,spark-shade-unbundle-avro"
skipModules: ""
- scala: "scala-2.12"
spark: "spark3.1.x"
skipModules: "!hudi-spark-datasource/hudi-spark3"
- scala: "scala-2.12"
spark: "spark3.1.x,spark-shade-unbundle-avro"
skipModules: "!hudi-spark-datasource/hudi-spark3"
- scala: "scala-2.12"
spark: "spark3"
- scala: "scala-2.12"
Expand All @@ -40,4 +44,5 @@ jobs:
env:
SCALA_PROFILE: ${{ matrix.scala }}
SPARK_PROFILE: ${{ matrix.spark }}
run: mvn install -P "$SCALA_PROFILE,$SPARK_PROFILE" -DskipTests=true -Dmaven.javadoc.skip=true -B -V
SKIP_MODULES: ${{ matrix.skipModules }}
run: mvn install -P "$SCALA_PROFILE,$SPARK_PROFILE" -pl "$SKIP_MODULES" -DskipTests=true -Dmaven.javadoc.skip=true -B -V
Original file line number Diff line number Diff line change
Expand Up @@ -53,8 +53,18 @@ object HoodieSparkUtils extends SparkAdapterSupport {

def isSpark3_0: Boolean = SPARK_VERSION.startsWith("3.0")

def isSpark3_1: Boolean = SPARK_VERSION.startsWith("3.1")

def isSpark3_2: Boolean = SPARK_VERSION.startsWith("3.2")

def beforeSpark3_2(): Boolean = {
if (isSpark2 || isSpark3_0 || isSpark3_1) {
true
} else {
false
}
}

def getMetaSchema: StructType = {
StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => {
StructField(col, StringType, nullable = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,19 +18,22 @@

package org.apache.spark.sql.hudi

import org.apache.hudi.HoodieSparkUtils.sparkAdapter
import org.apache.hudi.client.utils.SparkRowSerDe

import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, SubqueryAlias}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, SparkParsePartitionUtil}
import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.{Row, SparkSession}

import java.util.Locale

/**
* An interface to adapter the difference between spark2 and spark3
* in some spark related class.
Expand Down Expand Up @@ -99,4 +102,35 @@ trait SparkAdapter extends Serializable {
*/
def getFilePartitions(sparkSession: SparkSession, partitionedFiles: Seq[PartitionedFile],
maxSplitBytes: Long): Seq[FilePartition]

def isHoodieTable(table: LogicalPlan, spark: SparkSession): Boolean = {
tripAlias(table) match {
case LogicalRelation(_, _, Some(tbl), _) => isHoodieTable(tbl)
case relation: UnresolvedRelation =>
isHoodieTable(toTableIdentifier(relation), spark)
case _=> false
}
}

def isHoodieTable(map: java.util.Map[String, String]): Boolean = {
map.getOrDefault("provider", "").equals("hudi")
}

def isHoodieTable(table: CatalogTable): Boolean = {
table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"
}

def isHoodieTable(tableId: TableIdentifier, spark: SparkSession): Boolean = {
val table = spark.sessionState.catalog.getTableMetadata(tableId)
isHoodieTable(table)
}

def tripAlias(plan: LogicalPlan): LogicalPlan = {
plan match {
case SubqueryAlias(_, relation: LogicalPlan) =>
tripAlias(relation)
case other =>
other
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ class IncrementalRelation(val sqlContext: SQLContext,

if (doFullTableScan) {
val hudiDF = sqlContext.read
.format("hudi")
.format("hudi_v1")
.schema(usedSchema)
.load(basePath)
.filter(String.format("%s > '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, //Notice the > in place of >= because we are working with optParam instead of first commit > optParam
Expand All @@ -208,7 +208,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
} else {
if (metaBootstrapFileIdToFullPath.nonEmpty) {
df = sqlContext.sparkSession.read
.format("hudi")
.format("hudi_v1")
.schema(usedSchema)
.option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(","))
.load()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedRelation}
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression}
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.internal.StaticSQLConf
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.types.{DataType, NullType, StringType, StructField, StructType}
import org.apache.spark.sql.{Column, DataFrame, SparkSession}

import java.net.URI
Expand All @@ -54,24 +54,6 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
override def get() = new SimpleDateFormat("yyyy-MM-dd")
})

def isHoodieTable(table: CatalogTable): Boolean = {
table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"
}

def isHoodieTable(tableId: TableIdentifier, spark: SparkSession): Boolean = {
val table = spark.sessionState.catalog.getTableMetadata(tableId)
isHoodieTable(table)
}

def isHoodieTable(table: LogicalPlan, spark: SparkSession): Boolean = {
tripAlias(table) match {
case LogicalRelation(_, _, Some(tbl), _) => isHoodieTable(tbl)
case relation: UnresolvedRelation =>
isHoodieTable(sparkAdapter.toTableIdentifier(relation), spark)
case _=> false
}
}

def getTableIdentifier(table: LogicalPlan): TableIdentifier = {
table match {
case SubqueryAlias(name, _) => sparkAdapter.toTableIdentifier(name)
Expand Down Expand Up @@ -200,14 +182,29 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
getTableLocation(table, spark)
}

def getTableLocation(properties: Map[String, String], identifier: TableIdentifier, sparkSession: SparkSession): String = {
val location: Option[String] = Some(properties.getOrElse("location", ""))
val isManaged = location.isEmpty || location.get.isEmpty
val uri = if (isManaged) {
Some(sparkSession.sessionState.catalog.defaultTablePath(identifier))
} else {
Some(new Path(location.get).toUri)
}
getTableLocation(uri, identifier, sparkSession)
}

def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
val uri = table.storage.locationUri.orElse {
Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
}
getTableLocation(uri, table.identifier, sparkSession)
}

def getTableLocation(uri: Option[URI], identifier: TableIdentifier, sparkSession: SparkSession): String = {
val conf = sparkSession.sessionState.newHadoopConf()
uri.map(makePathQualified(_, conf))
.map(removePlaceHolder)
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${identifier}"))
}

private def removePlaceHolder(path: String): String = {
Expand Down Expand Up @@ -316,4 +313,12 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
def columnEqual(field: StructField, other: StructField, resolver: Resolver): Boolean = {
resolver(field.name, other.name) && field.dataType == other.dataType
}

def castIfNeeded(child: Expression, dataType: DataType, conf: SQLConf): Expression = {
child match {
case Literal(nul, NullType) => Literal(nul, dataType)
case _ => if (child.dataType != dataType)
Cast(child, dataType, Option(conf.sessionLocalTimeZone)) else child
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,8 @@ case class AlterHoodieTableAddColumnsCommand(
s" table columns is: [${hoodieCatalogTable.tableSchemaWithoutMetaFields.fieldNames.mkString(",")}]")
}
// Get the new schema
val newSqlSchema = StructType(tableSchema.fields ++ colsToAdd)
val rearrangedSchema = hoodieCatalogTable.dataSchema ++ colsToAdd ++ hoodieCatalogTable.partitionSchema
val newSqlSchema = StructType(rearrangedSchema)
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableId.table)
val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,8 @@ package org.apache.spark.sql.hudi

import org.apache.hudi.SparkAdapterSupport
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.expressions.{And, Cast, Expression, Literal}
import org.apache.spark.sql.catalyst.expressions.{And, Expression}
import org.apache.spark.sql.catalyst.plans.logical.{MergeIntoTable, SubqueryAlias}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, NullType}

object HoodieSqlUtils extends SparkAdapterSupport {

Expand Down Expand Up @@ -50,12 +48,4 @@ object HoodieSqlUtils extends SparkAdapterSupport {
case exp => Seq(exp)
}
}

def castIfNeeded(child: Expression, dataType: DataType, conf: SQLConf): Expression = {
child match {
case Literal(nul, NullType) => Literal(nul, dataType)
case _ => if (child.dataType != dataType)
Cast(child, dataType, Option(conf.sessionLocalTimeZone)) else child
}
}
}
Loading

0 comments on commit 0db1e97

Please sign in to comment.