diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml
index fc79fefd20658..60553c950348a 100644
--- a/.github/workflows/bot.yml
+++ b/.github/workflows/bot.yml
@@ -18,16 +18,22 @@ 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,spark3.0.x"
+ skipModules: "!hudi-spark-datasource/hudi-spark3"
- scala: "scala-2.12"
spark: "spark3,spark3.0.x,spark-shade-unbundle-avro"
+ skipModules: "!hudi-spark-datasource/hudi-spark3"
- scala: "scala-2.12"
spark: "spark3,spark3.1.x"
+ skipModules: "!hudi-spark-datasource/hudi-spark3"
- scala: "scala-2.12"
spark: "spark3,spark3.1.x,spark-shade-unbundle-avro"
+ skipModules: "!hudi-spark-datasource/hudi-spark3"
- scala: "scala-2.12"
spark: "spark3"
- scala: "scala-2.12"
@@ -44,4 +50,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
\ No newline at end of file
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
index a93331ee20d48..7803c184e0a6f 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
@@ -20,16 +20,19 @@ package org.apache.spark.sql.hudi
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.SparkParsePartitionUtil
+import org.apache.spark.sql.execution.datasources.{LogicalRelation, 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.
@@ -44,12 +47,12 @@ trait SparkAdapter extends Serializable {
/**
* Convert a AliasIdentifier to TableIdentifier.
*/
- def toTableIdentify(aliasId: AliasIdentifier): TableIdentifier
+ def toTableIdentifier(aliasId: AliasIdentifier): TableIdentifier
/**
* Convert a UnresolvedRelation to TableIdentifier.
*/
- def toTableIdentify(relation: UnresolvedRelation): TableIdentifier
+ def toTableIdentifier(relation: UnresolvedRelation): TableIdentifier
/**
* Create Join logical plan.
@@ -92,4 +95,31 @@ trait SparkAdapter extends Serializable {
* ParserInterface#parseMultipartIdentifier is supported since spark3, for spark2 this should not be called.
*/
def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String]
+
+ 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 tripAlias(plan: LogicalPlan): LogicalPlan = {
+ plan match {
+ case SubqueryAlias(_, relation: LogicalPlan) =>
+ tripAlias(relation)
+ case other =>
+ other
+ }
+ }
+
+ 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)
+ }
}
diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml
index caa218a70c663..790fd50b63a36 100644
--- a/hudi-spark-datasource/hudi-spark-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark-common/pom.xml
@@ -170,9 +170,51 @@
${project.version}
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+
+
+ javax.servlet
+ *
+
+
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+
+
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+
+
org.apache.spark
spark-sql_${scala.binary.version}
+ tests
+ test
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ tests
+ test
+
+
+ org.apache.spark
+ spark-catalyst_${scala.binary.version}
+ tests
+ test
+
+
+
+
+ org.apache.spark
+ spark-avro_${scala.binary.version}
+ provided
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncClusteringService.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/async/SparkStreamingAsyncClusteringService.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncClusteringService.java
rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/async/SparkStreamingAsyncClusteringService.java
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/sql/IExpressionEvaluator.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/IExpressionEvaluator.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/sql/IExpressionEvaluator.java
rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/IExpressionEvaluator.java
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/sql/InsertMode.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/sql/InsertMode.java
rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java
diff --git a/hudi-spark-datasource/hudi-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark-common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
rename to hudi-spark-datasource/hudi-spark-common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala
index 9b437f5451b11..a57691f9f5cf4 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala
@@ -177,7 +177,7 @@ class DefaultSource extends RelationProvider
outputMode)
}
- override def shortName(): String = "hudi"
+ override def shortName(): String = "hudi_v1"
private def getBaseFileOnlyView(useHoodieFileIndex: Boolean,
sqlContext: SQLContext,
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala
index a522db6afc6f1..9d7b413565cd7 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala
@@ -18,13 +18,13 @@
package org.apache.hudi
-import org.apache.spark.{Partition, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.{Partition, TaskContext}
class HoodieBootstrapRDD(@transient spark: SparkSession,
dataReadFunction: PartitionedFile => Iterator[Any],
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala
index b1ab83a94cc9d..5cada7b7edcc0 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala
@@ -50,7 +50,7 @@ import scala.collection.JavaConverters._
* else read data from tablePath using HoodiFileIndex.
* @param metaClient Hoodie table meta client
* @param optParams DataSource options passed by the user
- */
+ */
class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext,
val userSchema: StructType,
val globPaths: Option[Seq[Path]],
@@ -116,7 +116,7 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext,
filters = if (requiredSkeletonSchema.isEmpty) filters else Seq() ,
options = Map.empty,
hadoopConf = _sqlContext.sparkSession.sessionState.newHadoopConf()
- )
+ )
val skeletonReadFunction = new ParquetFileFormat()
.buildReaderWithPartitionValues(
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
index 572a97e9dbe3d..00e6be627b3be 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
@@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory}
import org.apache.spark.sql.hudi.DataSkippingUtils.createColumnStatsIndexFilterExpr
-import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.sql.{AnalysisException, Column, SparkSession}
@@ -87,11 +87,11 @@ case class HoodieFileIndex(
private val tableType = metaClient.getTableType
private val specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
- .map(HoodieSqlUtils.formatQueryInstant)
+ .map(HoodieSqlCommonUtils.formatQueryInstant)
/**
* Get all completeCommits.
- */
+ */
lazy val completedCommits = metaClient.getCommitsTimeline
.filterCompletedInstants().getInstants.iterator().toList.map(_.getTimestamp)
@@ -403,7 +403,7 @@ case class HoodieFileIndex(
cachedAllInputFileSlices = partitionFiles.map(p => {
val latestSlices = if (latestInstant.isPresent) {
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p._1.partitionPath, queryInstant.get)
- .iterator().asScala.toSeq
+ .iterator().asScala.toSeq
} else {
Seq()
}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala
similarity index 92%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala
index ffe2c92829698..226fb01f43f90 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala
@@ -119,7 +119,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
}
private def logFileIterator(split: HoodieMergeOnReadFileSplit,
- config: Configuration): Iterator[InternalRow] =
+ config: Configuration): Iterator[InternalRow] =
new Iterator[InternalRow] with Closeable {
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
@@ -168,8 +168,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
}
private def skipMergeFileIterator(split: HoodieMergeOnReadFileSplit,
- baseFileIterator: Iterator[InternalRow],
- config: Configuration): Iterator[InternalRow] =
+ baseFileIterator: Iterator[InternalRow],
+ config: Configuration): Iterator[InternalRow] =
new Iterator[InternalRow] with Closeable {
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
@@ -225,8 +225,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
}
private def payloadCombineFileIterator(split: HoodieMergeOnReadFileSplit,
- baseFileIterator: Iterator[InternalRow],
- config: Configuration): Iterator[InternalRow] =
+ baseFileIterator: Iterator[InternalRow],
+ config: Configuration): Iterator[InternalRow] =
new Iterator[InternalRow] with Closeable {
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
@@ -350,23 +350,23 @@ private object HoodieMergeOnReadRDD {
def scanLog(split: HoodieMergeOnReadFileSplit, logSchema: Schema, config: Configuration): HoodieMergedLogRecordScanner = {
val fs = FSUtils.getFs(split.tablePath, config)
HoodieMergedLogRecordScanner.newBuilder()
- .withFileSystem(fs)
- .withBasePath(split.tablePath)
- .withLogFilePaths(split.logPaths.get.asJava)
- .withReaderSchema(logSchema)
- .withLatestInstantTime(split.latestCommit)
- .withReadBlocksLazily(
- Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
- HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
- .getOrElse(false))
- .withReverseReader(false)
- .withBufferSize(
- config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
- HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
- .withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes)
- .withSpillableMapBasePath(
- config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
- HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
- .build()
+ .withFileSystem(fs)
+ .withBasePath(split.tablePath)
+ .withLogFilePaths(split.logPaths.get.asJava)
+ .withReaderSchema(logSchema)
+ .withLatestInstantTime(split.latestCommit)
+ .withReadBlocksLazily(
+ Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
+ HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
+ .getOrElse(false))
+ .withReverseReader(false)
+ .withBufferSize(
+ config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
+ HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
+ .withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes)
+ .withSpillableMapBasePath(
+ config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
+ HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
+ .build()
}
}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
index d35cd49a482f4..4321621bd2ebd 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
@@ -177,7 +177,7 @@ object HoodieSparkSqlWriter {
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
null, path, tblName,
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
- .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
+ .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
asyncCompactionTriggerFn.get.apply(client)
@@ -487,9 +487,9 @@ object HoodieSparkSqlWriter {
val syncHiveSuccess =
if (hiveSyncEnabled || metaSyncEnabled) {
metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
- } else {
- true
- }
+ } else {
+ true
+ }
(syncHiveSuccess, common.util.Option.ofNullable(instantTime))
}
@@ -619,7 +619,7 @@ object HoodieSparkSqlWriter {
tableConfig: HoodieTableConfig,
jsc: JavaSparkContext,
tableInstantInfo: TableInstantInfo
- ): (Boolean, common.util.Option[java.lang.String], common.util.Option[java.lang.String]) = {
+ ): (Boolean, common.util.Option[java.lang.String], common.util.Option[java.lang.String]) = {
if(writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors).isEmpty()) {
log.info("Proceeding to commit the write.")
val metaMap = parameters.filter(kv =>
@@ -715,7 +715,7 @@ object HoodieSparkSqlWriter {
}
private def mergeParamsAndGetHoodieConfig(optParams: Map[String, String],
- tableConfig: HoodieTableConfig): (Map[String, String], HoodieConfig) = {
+ tableConfig: HoodieTableConfig): (Map[String, String], HoodieConfig) = {
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(optParams)
val mergedParams = mutable.Map.empty ++ HoodieWriterUtils.parametersWithWriteDefaults(translatedOptions)
if (!mergedParams.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
index 8d8ebfa7e54f1..ab1f61c67cc5b 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
@@ -129,9 +129,9 @@ class HoodieStreamingSink(sqlContext: SQLContext,
case Success((false, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) =>
log.error(s"Micro batch id=$batchId ended up with errors"
+ (commitOps.isPresent match {
- case true => s" for commit=${commitOps.get()}"
- case _ => s""
- }))
+ case true => s" for commit=${commitOps.get()}"
+ case _ => s""
+ }))
if (ignoreFailedBatch) {
log.info(s"Ignore the errors and move on streaming as per " +
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.key} configuration")
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
similarity index 97%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
index 282de54f65259..9b7da633a280a 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
@@ -45,7 +45,7 @@ object HoodieWriterUtils {
*
* @param parameters
* @return
- */
+ */
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala
val props = new Properties()
@@ -123,7 +123,7 @@ object HoodieWriterUtils {
* Detects conflicts between new parameters and existing table configurations
*/
def validateTableConfig(spark: SparkSession, params: Map[String, String],
- tableConfig: HoodieConfig): Unit = {
+ tableConfig: HoodieConfig): Unit = {
val resolver = spark.sessionState.conf.resolver
val diffConfigs = StringBuilder.newBuilder
params.foreach { case (key, value) =>
@@ -137,21 +137,21 @@ object HoodieWriterUtils {
val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null)
val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)
if (null != datasourceRecordKey && null != tableConfigRecordKey
- && datasourceRecordKey != tableConfigRecordKey) {
+ && datasourceRecordKey != tableConfigRecordKey) {
diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n")
}
val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null)
val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)
if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey
- && datasourcePreCombineKey != tableConfigPreCombineKey) {
+ && datasourcePreCombineKey != tableConfigPreCombineKey) {
diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n")
}
val datasourceKeyGen = getOriginKeyGenerator(params)
val tableConfigKeyGen = tableConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)
if (null != datasourceKeyGen && null != tableConfigKeyGen
- && datasourceKeyGen != tableConfigKeyGen) {
+ && datasourceKeyGen != tableConfigKeyGen) {
diffConfigs.append(s"KeyGenerator:\t$datasourceKeyGen\t$tableConfigKeyGen\n")
}
}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala
similarity index 92%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala
index 958a15eeb7adb..f01c1cf76fd09 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala
@@ -43,7 +43,7 @@ import scala.collection.mutable
*
* Implemented for Copy_on_write storage.
*
- */
+ */
class IncrementalRelation(val sqlContext: SQLContext,
val optParams: Map[String, String],
val userSchema: StructType,
@@ -161,21 +161,21 @@ class IncrementalRelation(val sqlContext: SQLContext,
if (metaBootstrapFileIdToFullPath.nonEmpty) {
df = sqlContext.sparkSession.read
- .format("hudi")
- .schema(usedSchema)
- .option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(","))
- .load()
+ .format("hudi_v1")
+ .schema(usedSchema)
+ .option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(","))
+ .load()
}
if (regularFileIdToFullPath.nonEmpty)
{
df = df.union(sqlContext.read.options(sOpts)
- .schema(usedSchema)
- .parquet(filteredRegularFullPaths.toList: _*)
- .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
- commitsToReturn.head.getTimestamp))
- .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
- commitsToReturn.last.getTimestamp)))
+ .schema(usedSchema)
+ .parquet(filteredRegularFullPaths.toList: _*)
+ .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
+ commitsToReturn.head.getTimestamp))
+ .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
+ commitsToReturn.last.getTimestamp)))
}
filters.foldLeft(df)((e, f) => e.filter(f)).rdd
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
index b4a9800d994b9..6a17bda4f2376 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
@@ -42,7 +42,7 @@ import scala.collection.JavaConversions._
* Experimental.
* Relation, that implements the Hoodie incremental view for Merge On Read table.
*
- */
+ */
class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
val optParams: Map[String, String],
val userSchema: StructType,
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
index a1d857c948c17..b30e5731021ea 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile}
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
-import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import org.apache.spark.sql.{Row, SQLContext}
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
import org.apache.spark.sql.types.StructType
@@ -99,7 +99,7 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
override def needConversion: Boolean = false
private val specifiedQueryInstant = optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
- .map(HoodieSqlUtils.formatQueryInstant)
+ .map(HoodieSqlCommonUtils.formatQueryInstant)
override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}")
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/package.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/package.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/package.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/package.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
similarity index 87%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
index 21a60087f0f9b..ea8d91d680d48 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
@@ -30,8 +30,8 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.types.{StructField, StructType}
import java.util.{Locale, Properties}
@@ -62,7 +62,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
* hoodie table's location.
* if create managed hoodie table, use `catalog.defaultTablePath`.
*/
- val tableLocation: String = HoodieSqlUtils.getTableLocation(table, spark)
+ val tableLocation: String = HoodieSqlCommonUtils.getTableLocation(table, spark)
/**
* A flag to whether the hoodie table exists.
@@ -124,7 +124,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
/**
* The schema without hoodie meta fields
*/
- lazy val tableSchemaWithoutMetaFields: StructType = HoodieSqlUtils.removeMetaFields(tableSchema)
+ lazy val tableSchemaWithoutMetaFields: StructType = HoodieSqlCommonUtils.removeMetaFields(tableSchema)
/**
* The schema of data fields
@@ -136,7 +136,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
/**
* The schema of data fields not including hoodie meta fields
*/
- lazy val dataSchemaWithoutMetaFields: StructType = HoodieSqlUtils.removeMetaFields(dataSchema)
+ lazy val dataSchemaWithoutMetaFields: StructType = HoodieSqlCommonUtils.removeMetaFields(dataSchema)
/**
* The schema of partition fields
@@ -146,7 +146,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
/**
* All the partition paths
*/
- def getAllPartitionPaths: Seq[String] = HoodieSqlUtils.getAllPartitionPaths(spark, table)
+ def getAllPartitionPaths: Seq[String] = HoodieSqlCommonUtils.getAllPartitionPaths(spark, table)
/**
* Check if table is a partitioned table
@@ -213,7 +213,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
case (CatalogTableType.MANAGED, true) =>
throw new AnalysisException(s"Can not create the managed table('$catalogTableName')" +
- s". The associated location('$tableLocation') already exists.")
+ s". The associated location('$tableLocation') already exists.")
}
HoodieOptionConfig.validateTable(spark, finalSchema,
HoodieOptionConfig.mappingTableConfigToSqlOption(tableConfigs))
@@ -228,23 +228,23 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
}
private def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean,
- originTableConfig: Map[String, String] = Map.empty): Map[String, String] = {
+ originTableConfig: Map[String, String] = Map.empty): Map[String, String] = {
val extraConfig = mutable.Map.empty[String, String]
if (isTableExists) {
val allPartitionPaths = getAllPartitionPaths
if (originTableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)) {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) =
- originTableConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
+ originTableConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
} else {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) =
- String.valueOf(isHiveStyledPartitioning(allPartitionPaths, table))
+ String.valueOf(isHiveStyledPartitioning(allPartitionPaths, table))
}
if (originTableConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)) {
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) =
- originTableConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
+ originTableConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
} else {
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) =
- String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table))
+ String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table))
}
} else {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true"
@@ -253,8 +253,8 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) {
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) =
- HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(
- originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key))
+ HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(
+ originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key))
} else {
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[ComplexKeyGenerator].getCanonicalName
}
@@ -263,7 +263,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
// This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema
private def verifyDataSchema(tableIdentifier: TableIdentifier, tableType: CatalogTableType,
- dataSchema: Seq[StructField]): Unit = {
+ dataSchema: Seq[StructField]): Unit = {
if (tableType != CatalogTableType.VIEW) {
val invalidChars = Seq(",", ":", ";")
def verifyNestedColumnNames(schema: StructType): Unit = schema.foreach { f =>
@@ -272,8 +272,8 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
case _ if invalidChars.exists(f.name.contains) =>
val invalidCharsString = invalidChars.map(c => s"'$c'").mkString(", ")
val errMsg = "Cannot create a table having a nested column whose name contains " +
- s"invalid characters ($invalidCharsString) in Hive metastore. Table: $tableIdentifier; " +
- s"Column: ${f.name}"
+ s"invalid characters ($invalidCharsString) in Hive metastore. Table: $tableIdentifier; " +
+ s"Column: ${f.name}"
throw new AnalysisException(errMsg)
case _ =>
}
@@ -284,7 +284,7 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
// Checks top-level column names
case _ if f.name.contains(",") =>
throw new AnalysisException("Cannot create a table having a column whose name " +
- s"contains commas in Hive metastore. Table: $tableIdentifier; Column: ${f.name}")
+ s"contains commas in Hive metastore. Table: $tableIdentifier; Column: ${f.name}")
// Checks nested column names
case st: StructType =>
verifyNestedColumnNames(st)
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hive/HiveClientUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hive/HiveClientUtils.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hive/HiveClientUtils.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hive/HiveClientUtils.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala
index e5d6f525bac4b..6ac5d9d685dc0 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala
@@ -267,9 +267,9 @@ object DataSkippingUtils extends Logging {
}
/**
- * read parquet files concurrently by local.
- * this method is mush faster than spark
- */
+ * read parquet files concurrently by local.
+ * this method is mush faster than spark
+ */
def readParquetFile(spark: SparkSession, indexFiles: Seq[FileStatus], filters: Seq[Filter] = Nil, schemaOpts: Option[StructType] = None): Set[String] = {
val hadoopConf = spark.sparkContext.hadoopConfiguration
val partitionedFiles = indexFiles.map(f => PartitionedFile(InternalRow.empty, f.getPath.toString, 0, f.getLen))
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
index bc9f14978c292..e3388e221a972 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.DataSourceWriteOptions
-import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
+import org.apache.hudi.common.model.DefaultHoodieRecordPayload
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.common.util.ValidationUtils
import org.apache.spark.sql.SparkSession
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
new file mode 100644
index 0000000000000..7f5844b16aa5c
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
@@ -0,0 +1,316 @@
+/*
+ * 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 scala.collection.JavaConverters._
+import java.net.URI
+import java.util.{Date, Locale, Properties}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.config.DFSPropertiesConfiguration
+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, HoodieInstantTimeGenerator}
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.{Column, DataFrame, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
+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.{SQLConf, StaticSQLConf}
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.types.{DataType, NullType, StringType, StructField, StructType}
+
+import java.text.SimpleDateFormat
+
+import scala.collection.immutable.Map
+
+object HoodieSqlCommonUtils extends SparkAdapterSupport {
+ // NOTE: {@code SimpleDataFormat} is NOT thread-safe
+ // TODO replace w/ DateTimeFormatter
+ private val defaultDateFormat =
+ ThreadLocal.withInitial(new java.util.function.Supplier[SimpleDateFormat] {
+ 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 getTableIdentify(table: LogicalPlan): TableIdentifier = {
+ table match {
+ case SubqueryAlias(name, _) => sparkAdapter.toTableIdentifier(name)
+ case _ => throw new IllegalArgumentException(s"Illegal table: $table")
+ }
+ }
+
+ def getTableSqlSchema(metaClient: HoodieTableMetaClient,
+ includeMetadataFields: Boolean = false): Option[StructType] = {
+ val schemaResolver = new TableSchemaResolver(metaClient)
+ val avroSchema = try Some(schemaResolver.getTableAvroSchema(includeMetadataFields))
+ catch {
+ case _: Throwable => None
+ }
+ avroSchema.map(AvroConversionUtils.convertAvroSchemaToStructType)
+ }
+
+ 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 ++ table.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"
+ * when writing data to hudi table by spark sql by default.
+ * If the exist table is a non-hive-styled partitioned table, we should
+ * disable the "hoodie.datasource.write.hive_style_partitioning" when
+ * merge or update the table. Or else, we will get an incorrect merge result
+ * as the partition path mismatch.
+ */
+ def isHiveStyledPartitioning(partitionPaths: Seq[String], table: CatalogTable): Boolean = {
+ if (table.partitionColumnNames.nonEmpty) {
+ val isHiveStylePartitionPath = (path: String) => {
+ val fragments = path.split("/")
+ if (fragments.size != table.partitionColumnNames.size) {
+ false
+ } else {
+ fragments.zip(table.partitionColumnNames).forall {
+ case (pathFragment, partitionColumn) => pathFragment.startsWith(s"$partitionColumn=")
+ }
+ }
+ }
+ partitionPaths.forall(isHiveStylePartitionPath)
+ } else {
+ true
+ }
+ }
+
+ /**
+ * Determine whether URL encoding is enabled
+ */
+ def isUrlEncodeEnabled(partitionPaths: Seq[String], table: CatalogTable): Boolean = {
+ if (table.partitionColumnNames.nonEmpty) {
+ partitionPaths.forall(partitionPath => partitionPath.split("/").length == table.partitionColumnNames.size)
+ } else {
+ false
+ }
+ }
+
+ private def tripAlias(plan: LogicalPlan): LogicalPlan = {
+ plan match {
+ case SubqueryAlias(_, relation: LogicalPlan) =>
+ tripAlias(relation)
+ case other =>
+ other
+ }
+ }
+
+ /**
+ * Add the hoodie meta fields to the schema.
+ * @param schema
+ * @return
+ */
+ def addMetaFields(schema: StructType): StructType = {
+ val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala
+ // filter the meta field to avoid duplicate field.
+ val dataFields = schema.fields.filterNot(f => metaFields.contains(f.name))
+ val fields = metaFields.map(StructField(_, StringType)) ++ dataFields
+ StructType(fields)
+ }
+
+ private lazy val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala.toSet
+
+ /**
+ * Remove the meta fields from the schema.
+ * @param schema
+ * @return
+ */
+ def removeMetaFields(schema: StructType): StructType = {
+ StructType(schema.fields.filterNot(f => isMetaField(f.name)))
+ }
+
+ def isMetaField(name: String): Boolean = {
+ metaFields.contains(name)
+ }
+
+ def removeMetaFields(df: DataFrame): DataFrame = {
+ val withoutMetaColumns = df.logicalPlan.output
+ .filterNot(attr => isMetaField(attr.name))
+ .map(new Column(_))
+ if (withoutMetaColumns.length != df.logicalPlan.output.size) {
+ df.select(withoutMetaColumns: _*)
+ } else {
+ df
+ }
+ }
+
+ def removeMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = {
+ attrs.filterNot(attr => isMetaField(attr.name))
+ }
+
+ /**
+ * Get the table location.
+ * @param tableId
+ * @param spark
+ * @return
+ */
+ def getTableLocation(tableId: TableIdentifier, spark: SparkSession): String = {
+ val table = spark.sessionState.catalog.getTableMetadata(tableId)
+ getTableLocation(table, spark)
+ }
+
+ def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
+ val uri = if (table.tableType == CatalogTableType.MANAGED && isHoodieTable(table)) {
+ Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
+ } else {
+ table.storage.locationUri
+ }
+ val conf = sparkSession.sessionState.newHadoopConf()
+ uri.map(makePathQualified(_, conf))
+ .map(removePlaceHolder)
+ .getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
+ }
+
+ private def removePlaceHolder(path: String): String = {
+ if (path == null || path.length == 0) {
+ path
+ } else if (path.endsWith("-__PLACEHOLDER__")) {
+ path.substring(0, path.length() - 16)
+ } else {
+ path
+ }
+ }
+
+ def makePathQualified(path: URI, hadoopConf: Configuration): String = {
+ val hadoopPath = new Path(path)
+ val fs = hadoopPath.getFileSystem(hadoopConf)
+ fs.makeQualified(hadoopPath).toUri.toString
+ }
+
+ /**
+ * Check if the hoodie.properties exists in the table path.
+ */
+ def tableExistsInPath(tablePath: String, conf: Configuration): Boolean = {
+ val basePath = new Path(tablePath)
+ val fs = basePath.getFileSystem(conf)
+ val metaPath = new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)
+ fs.exists(metaPath)
+ }
+
+ 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
+ }
+ }
+
+ /**
+ * Split the expression to a sub expression seq by the AND operation.
+ * @param expression
+ * @return
+ */
+ def splitByAnd(expression: Expression): Seq[Expression] = {
+ expression match {
+ case And(left, right) =>
+ splitByAnd(left) ++ splitByAnd(right)
+ case exp => Seq(exp)
+ }
+ }
+
+ /**
+ * Append the spark config and table options to the baseConfig.
+ */
+ def withSparkConf(spark: SparkSession, options: Map[String, String])
+ (baseConfig: Map[String, String] = Map.empty): Map[String, String] = {
+ baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority
+ (spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options))
+ .filterKeys(_.startsWith("hoodie."))
+ }
+
+ def isEnableHive(sparkSession: SparkSession): Boolean =
+ "hive" == sparkSession.sessionState.conf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION)
+
+ /**
+ * Convert different query instant time format to the commit time format.
+ * Currently we support three kinds of instant time format for time travel query:
+ * 1、yyyy-MM-dd HH:mm:ss
+ * 2、yyyy-MM-dd
+ * This will convert to 'yyyyMMdd000000'.
+ * 3、yyyyMMddHHmmss
+ */
+ def formatQueryInstant(queryInstant: String): String = {
+ val instantLength = queryInstant.length
+ if (instantLength == 19 || instantLength == 23) { // for yyyy-MM-dd HH:mm:ss[.SSS]
+ HoodieInstantTimeGenerator.getInstantForDateString(queryInstant)
+ } else if (instantLength == HoodieInstantTimeGenerator.SECS_INSTANT_ID_LENGTH
+ || instantLength == HoodieInstantTimeGenerator.MILLIS_INSTANT_ID_LENGTH) { // for yyyyMMddHHmmss[SSS]
+ HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) // validate the format
+ queryInstant
+ } else if (instantLength == 10) { // for yyyy-MM-dd
+ HoodieActiveTimeline.formatDate(defaultDateFormat.get().parse(queryInstant))
+ } else {
+ throw new IllegalArgumentException(s"Unsupported query instant time format: $queryInstant,"
+ + s"Supported time format are: 'yyyy-MM-dd: HH:mm:ss.SSS' or 'yyyy-MM-dd' or 'yyyyMMddHHmmssSSS'")
+ }
+ }
+
+ def formatName(sparkSession: SparkSession, name: String): String = {
+ if (sparkSession.sessionState.conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
+ }
+
+ /**
+ * Check if this is a empty table path.
+ */
+ def isEmptyPath(tablePath: String, conf: Configuration): Boolean = {
+ val basePath = new Path(tablePath)
+ val fs = basePath.getFileSystem(conf)
+ if (fs.exists(basePath)) {
+ fs.listStatus(basePath).isEmpty
+ } else {
+ true
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala
similarity index 97%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala
index c6c08da1e69c3..a44a9b9aee2ce 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala
@@ -17,8 +17,6 @@
package org.apache.spark.sql.hudi.command
-import java.nio.charset.StandardCharsets
-
import org.apache.avro.Schema
import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
import org.apache.hudi.common.table.timeline.HoodieInstant.State
@@ -26,15 +24,15 @@ import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstan
import org.apache.hudi.common.util.{CommitUtils, Option}
import org.apache.hudi.table.HoodieSparkTable
import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, HoodieWriterUtils}
-
import org.apache.spark.api.java.JavaSparkContext
-import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types.{StructField, StructType}
import org.apache.spark.sql.util.SchemaUtils
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import java.nio.charset.StandardCharsets
import scala.collection.JavaConverters._
import scala.util.control.NonFatal
@@ -59,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 rearrangeSchema = hoodieCatalogTable.dataSchema ++ colsToAdd ++ hoodieCatalogTable.partitionSchema
+ val newSqlSchema = StructType(rearrangeSchema)
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableId.table)
val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace)
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
similarity index 95%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
index b69c686f99bf8..6ff2769bc6f7c 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala
@@ -18,17 +18,14 @@
package org.apache.spark.sql.hudi.command
import org.apache.avro.Schema
-
import org.apache.hudi.AvroConversionUtils
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.exception.HoodieException
-
-import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import scala.util.control.NonFatal
@@ -51,12 +48,12 @@ case class AlterHoodieTableChangeColumnCommand(
// Get the new schema
val newTableSchema = StructType(
hoodieCatalogTable.tableSchema.fields.map { field =>
- if (resolver(field.name, columnName)) {
- newColumn
- } else {
- field
- }
- })
+ if (resolver(field.name, columnName)) {
+ newColumn
+ } else {
+ field
+ }
+ })
val newDataSchema = StructType(
hoodieCatalogTable.dataSchema.fields.map { field =>
if (resolver(field.name, columnName)) {
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
similarity index 86%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
index 21f16275d1321..8f52e222e5d93 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
@@ -25,22 +25,21 @@ import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.MultiPartKeysValueExtractor
import org.apache.hudi.hive.ddl.HiveSyncMode
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter}
-
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.execution.command.DDLUtils
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession}
case class AlterHoodieTableDropPartitionCommand(
- tableIdentifier: TableIdentifier,
- specs: Seq[TablePartitionSpec],
- ifExists : Boolean,
- purge : Boolean,
- retainData : Boolean)
-extends HoodieLeafRunnableCommand {
+ tableIdentifier: TableIdentifier,
+ specs: Seq[TablePartitionSpec],
+ ifExists : Boolean,
+ purge : Boolean,
+ retainData : Boolean)
+ extends HoodieLeafRunnableCommand {
override def run(sparkSession: SparkSession): Seq[Row] = {
val fullTableName = s"${tableIdentifier.database}.${tableIdentifier.table}"
@@ -88,9 +87,9 @@ extends HoodieLeafRunnableCommand {
}
private def buildHoodieConfig(
- sparkSession: SparkSession,
- hoodieCatalogTable: HoodieCatalogTable,
- partitionsToDrop: String): Map[String, String] = {
+ sparkSession: SparkSession,
+ hoodieCatalogTable: HoodieCatalogTable,
+ partitionsToDrop: String): Map[String, String] = {
val partitionFields = hoodieCatalogTable.partitionFields.mkString(",")
val enableHive = isEnableHive(sparkSession)
withSparkConf(sparkSession, Map.empty) {
@@ -117,10 +116,10 @@ extends HoodieLeafRunnableCommand {
}
def normalizePartitionSpec[T](
- partitionSpec: Map[String, T],
- partColNames: Seq[String],
- tblName: String,
- resolver: Resolver): Map[String, T] = {
+ partitionSpec: Map[String, T],
+ partColNames: Seq[String],
+ tblName: String,
+ resolver: Resolver): Map[String, T] = {
val normalizedPartSpec = partitionSpec.toSeq.map { case (key, value) =>
val normalizedKey = partColNames.find(resolver(_, key)).getOrElse {
throw new AnalysisException(s"$key is not a valid partition column in table $tblName.")
@@ -146,8 +145,8 @@ extends HoodieLeafRunnableCommand {
}
def getPartitionPathToDrop(
- hoodieCatalogTable: HoodieCatalogTable,
- normalizedSpecs: Seq[Map[String, String]]): String = {
+ hoodieCatalogTable: HoodieCatalogTable,
+ normalizedSpecs: Seq[Map[String, String]]): String = {
val table = hoodieCatalogTable.table
val allPartitionPaths = hoodieCatalogTable.getAllPartitionPaths
val enableHiveStylePartitioning = isHiveStyledPartitioning(allPartitionPaths, table)
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala
similarity index 99%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala
index c7b5bdc202f65..5771e86db8ac9 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala
@@ -18,11 +18,10 @@
package org.apache.spark.sql.hudi.command
import org.apache.hudi.common.table.HoodieTableMetaClient
-
-import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.execution.command.AlterTableRenameCommand
+import org.apache.spark.sql.{Row, SparkSession}
/**
* Command for alter hudi table's table name.
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-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
similarity index 89%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
index 2608f9383601d..af3a6cb880156 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
@@ -18,21 +18,19 @@
package org.apache.spark.sql.hudi.command
import org.apache.hadoop.fs.Path
-
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.hadoop.HoodieParquetInputFormat
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
-
import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, TableAlreadyExistsException}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.hive.HiveClientUtils
import org.apache.spark.sql.hive.HiveExternalCatalog._
-import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive
+import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Row, SparkSession}
@@ -94,7 +92,7 @@ object CreateHoodieTableCommand {
}
def createTableInCatalog(sparkSession: SparkSession,
- hoodieCatalogTable: HoodieCatalogTable, ignoreIfExists: Boolean): Unit = {
+ hoodieCatalogTable: HoodieCatalogTable, ignoreIfExists: Boolean): Unit = {
val table = hoodieCatalogTable.table
assert(table.tableType != CatalogTableType.VIEW)
@@ -124,8 +122,8 @@ object CreateHoodieTableCommand {
table.storage.compressed,
storageProperties + ("path" -> path))
- val tablName = HoodieSqlUtils.formatName(sparkSession, table.identifier.table)
- val newDatabaseName = HoodieSqlUtils.formatName(sparkSession, table.identifier.database
+ val tablName = HoodieSqlCommonUtils.formatName(sparkSession, table.identifier.table)
+ val newDatabaseName = HoodieSqlCommonUtils.formatName(sparkSession, table.identifier.database
.getOrElse(catalog.getCurrentDatabase))
val newTableIdentifier = table.identifier
@@ -151,15 +149,15 @@ object CreateHoodieTableCommand {
}
/**
- * Create Hive table for hudi.
- * Firstly, do some check for the schema & table.
- * Secondly, append some table properties need for spark datasource table.
- * Thirdly, create hive table using the HiveClient.
- * @param table
- * @param sparkSession
- */
+ * Create Hive table for hudi.
+ * Firstly, do some check for the schema & table.
+ * Secondly, append some table properties need for spark datasource table.
+ * Thirdly, create hive table using the HiveClient.
+ * @param table
+ * @param sparkSession
+ */
private def createHiveDataSourceTable(sparkSession: SparkSession, table: CatalogTable,
- ignoreIfExists: Boolean): Unit = {
+ ignoreIfExists: Boolean): Unit = {
val dbName = table.identifier.database.get
// check database
val dbExists = sparkSession.sessionState.catalog.databaseExists(dbName)
@@ -183,7 +181,7 @@ object CreateHoodieTableCommand {
// This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#tableMetaToTableProps
private def tableMetaToTableProps(sparkConf: SparkConf, table: CatalogTable,
- schema: StructType): Map[String, String] = {
+ schema: StructType): Map[String, String] = {
val partitionColumns = table.partitionColumnNames
val bucketSpec = table.bucketSpec
@@ -231,13 +229,13 @@ object CreateHoodieTableCommand {
}
private def checkTableConfigEqual(
- originTableConfig: Map[String, String],
- newTableConfig: Map[String, String],
- configKey: String): Unit = {
+ originTableConfig: Map[String, String],
+ newTableConfig: Map[String, String],
+ configKey: String): Unit = {
if (originTableConfig.contains(configKey) && newTableConfig.contains(configKey)) {
assert(originTableConfig(configKey) == newTableConfig(configKey),
s"Table config: $configKey in the create table is: ${newTableConfig(configKey)}, is not the same with the value in " +
- s"hoodie.properties, which is: ${originTableConfig(configKey)}. Please keep the same.")
+ s"hoodie.properties, which is: ${originTableConfig(configKey)}. Please keep the same.")
}
}
}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala
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-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
similarity index 92%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
index 6a3eff8b4c356..4044e0c518e34 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
@@ -32,9 +32,9 @@ import org.apache.spark.sql.types.StringType
* Command for show hudi table's partitions.
*/
case class ShowHoodieTablePartitionsCommand(
- tableIdentifier: TableIdentifier,
- specOpt: Option[TablePartitionSpec])
-extends HoodieLeafRunnableCommand {
+ tableIdentifier: TableIdentifier,
+ specOpt: Option[TablePartitionSpec])
+ extends HoodieLeafRunnableCommand {
override val output: Seq[Attribute] = {
AttributeReference("partition", StringType, nullable = false)() :: Nil
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
similarity index 98%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
index e069df97aff55..a049c2e25430c 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
@@ -17,8 +17,6 @@
package org.apache.spark.sql.hudi.command
-import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS}
-
import org.apache.avro.generic.GenericRecord
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.util.PartitionPathEncodeUtils
@@ -27,7 +25,9 @@ import org.apache.hudi.keygen._
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{StructType, TimestampType}
-import org.joda.time.format.{DateTimeFormat, DateTimeFormatter}
+import org.joda.time.format.DateTimeFormat
+
+import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS}
/**
* A complex key generator for sql command which do some process for the
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala
similarity index 95%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala
index 12ec22499db7f..1277d630a5acf 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala
@@ -31,8 +31,8 @@ import scala.util.control.NonFatal
* Command for truncate hudi table.
*/
class TruncateHoodieTableCommand(
- tableIdentifier: TableIdentifier,
- partitionSpec: Option[TablePartitionSpec])
+ tableIdentifier: TableIdentifier,
+ partitionSpec: Option[TablePartitionSpec])
extends TruncateTableCommand(tableIdentifier, partitionSpec) {
override def run(sparkSession: SparkSession): Seq[Row] = {
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ValidateDuplicateKeyPayload.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ValidateDuplicateKeyPayload.scala
new file mode 100644
index 0000000000000..2619d1d9fe151
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ValidateDuplicateKeyPayload.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.avro.Schema
+import org.apache.avro.generic.{GenericRecord, IndexedRecord}
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord}
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.exception.HoodieDuplicateKeyException
+
+
+import java.util.Properties
+
+/**
+ * Validate the duplicate key for insert statement without enable the INSERT_DROP_DUPS_OPT
+ * config.
+ */
+class ValidateDuplicateKeyPayload(record: GenericRecord, orderingVal: Comparable[_])
+ extends DefaultHoodieRecordPayload(record, orderingVal) {
+
+ def this(record: HOption[GenericRecord]) {
+ this(if (record.isPresent) record.get else null, 0)
+ }
+
+ override def combineAndGetUpdateValue(currentValue: IndexedRecord,
+ schema: Schema, properties: Properties): HOption[IndexedRecord] = {
+ val key = currentValue.asInstanceOf[GenericRecord].get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
+ throw new HoodieDuplicateKeyException(key)
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionCodeGen.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala
rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala
diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml
index e20f4ad48c079..534691cf05c12 100644
--- a/hudi-spark-datasource/hudi-spark/pom.xml
+++ b/hudi-spark-datasource/hudi-spark/pom.xml
@@ -242,6 +242,18 @@
+
+ org.apache.hudi
+ ${hudi.spark.common.module}
+ ${project.version}
+
+
+ org.apache.hudi
+ *
+
+
+
+
log4j
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSparkSessionExtension.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSparkSessionExtension.scala
index a18a17f441de5..9cc95e6f9e90d 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSparkSessionExtension.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSparkSessionExtension.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.parser.HoodieCommonSqlParser
* The Hoodie SparkSessionExtension for extending the syntax and add the rules.
*/
class HoodieSparkSessionExtension extends (SparkSessionExtensions => Unit)
- with SparkAdapterSupport{
+ with SparkAdapterSupport {
override def apply(extensions: SparkSessionExtensions): Unit = {
extensions.injectParser { (session, parser) =>
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 3b6436ee2e60a..048ca4ec6e758 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
@@ -17,235 +17,12 @@
package org.apache.spark.sql.hudi
-import scala.collection.JavaConverters._
-import java.net.URI
-import java.util.{Date, Locale, Properties}
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-
-import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
-import org.apache.hudi.client.common.HoodieSparkEngineContext
-import org.apache.hudi.common.config.DFSPropertiesConfiguration
-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, HoodieInstantTimeGenerator}
-
-import org.apache.spark.sql.{Column, DataFrame, SparkSession}
+import org.apache.hudi.SparkAdapterSupport
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
-import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal}
-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
-
-import scala.collection.immutable.Map
+import org.apache.spark.sql.catalyst.expressions.{And, Expression}
+import org.apache.spark.sql.catalyst.plans.logical.{MergeIntoTable, SubqueryAlias}
object HoodieSqlUtils extends SparkAdapterSupport {
- // NOTE: {@code SimpleDataFormat} is NOT thread-safe
- // TODO replace w/ DateTimeFormatter
- private val defaultDateFormat =
- ThreadLocal.withInitial(new java.util.function.Supplier[SimpleDateFormat] {
- 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.toTableIdentify(relation), spark)
- case _=> false
- }
- }
-
- def getTableIdentify(table: LogicalPlan): TableIdentifier = {
- table match {
- case SubqueryAlias(name, _) => sparkAdapter.toTableIdentify(name)
- case _ => throw new IllegalArgumentException(s"Illegal table: $table")
- }
- }
-
- def getTableSqlSchema(metaClient: HoodieTableMetaClient,
- includeMetadataFields: Boolean = false): Option[StructType] = {
- val schemaResolver = new TableSchemaResolver(metaClient)
- val avroSchema = try Some(schemaResolver.getTableAvroSchema(includeMetadataFields))
- catch {
- case _: Throwable => None
- }
- avroSchema.map(AvroConversionUtils.convertAvroSchemaToStructType)
- }
-
- 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 ++ table.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"
- * when writing data to hudi table by spark sql by default.
- * If the exist table is a non-hive-styled partitioned table, we should
- * disable the "hoodie.datasource.write.hive_style_partitioning" when
- * merge or update the table. Or else, we will get an incorrect merge result
- * as the partition path mismatch.
- */
- def isHiveStyledPartitioning(partitionPaths: Seq[String], table: CatalogTable): Boolean = {
- if (table.partitionColumnNames.nonEmpty) {
- val isHiveStylePartitionPath = (path: String) => {
- val fragments = path.split("/")
- if (fragments.size != table.partitionColumnNames.size) {
- false
- } else {
- fragments.zip(table.partitionColumnNames).forall {
- case (pathFragment, partitionColumn) => pathFragment.startsWith(s"$partitionColumn=")
- }
- }
- }
- partitionPaths.forall(isHiveStylePartitionPath)
- } else {
- true
- }
- }
-
- /**
- * Determine whether URL encoding is enabled
- */
- def isUrlEncodeEnabled(partitionPaths: Seq[String], table: CatalogTable): Boolean = {
- if (table.partitionColumnNames.nonEmpty) {
- partitionPaths.forall(partitionPath => partitionPath.split("/").length == table.partitionColumnNames.size)
- } else {
- false
- }
- }
-
- private def tripAlias(plan: LogicalPlan): LogicalPlan = {
- plan match {
- case SubqueryAlias(_, relation: LogicalPlan) =>
- tripAlias(relation)
- case other =>
- other
- }
- }
-
- /**
- * Add the hoodie meta fields to the schema.
- * @param schema
- * @return
- */
- def addMetaFields(schema: StructType): StructType = {
- val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala
- // filter the meta field to avoid duplicate field.
- val dataFields = schema.fields.filterNot(f => metaFields.contains(f.name))
- val fields = metaFields.map(StructField(_, StringType)) ++ dataFields
- StructType(fields)
- }
-
- private lazy val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala.toSet
-
- /**
- * Remove the meta fields from the schema.
- * @param schema
- * @return
- */
- def removeMetaFields(schema: StructType): StructType = {
- StructType(schema.fields.filterNot(f => isMetaField(f.name)))
- }
-
- def isMetaField(name: String): Boolean = {
- metaFields.contains(name)
- }
-
- def removeMetaFields(df: DataFrame): DataFrame = {
- val withoutMetaColumns = df.logicalPlan.output
- .filterNot(attr => isMetaField(attr.name))
- .map(new Column(_))
- if (withoutMetaColumns.length != df.logicalPlan.output.size) {
- df.select(withoutMetaColumns: _*)
- } else {
- df
- }
- }
-
- def removeMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = {
- attrs.filterNot(attr => isMetaField(attr.name))
- }
-
- /**
- * Get the table location.
- * @param tableId
- * @param spark
- * @return
- */
- def getTableLocation(tableId: TableIdentifier, spark: SparkSession): String = {
- val table = spark.sessionState.catalog.getTableMetadata(tableId)
- getTableLocation(table, spark)
- }
-
- def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
- val uri = if (table.tableType == CatalogTableType.MANAGED && isHoodieTable(table)) {
- Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
- } else {
- table.storage.locationUri
- }
- val conf = sparkSession.sessionState.newHadoopConf()
- uri.map(makePathQualified(_, conf))
- .map(removePlaceHolder)
- .getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
- }
-
- private def removePlaceHolder(path: String): String = {
- if (path == null || path.length == 0) {
- path
- } else if (path.endsWith("-__PLACEHOLDER__")) {
- path.substring(0, path.length() - 16)
- } else {
- path
- }
- }
-
- def makePathQualified(path: URI, hadoopConf: Configuration): String = {
- val hadoopPath = new Path(path)
- val fs = hadoopPath.getFileSystem(hadoopConf)
- fs.makeQualified(hadoopPath).toUri.toString
- }
-
- /**
- * Check if the hoodie.properties exists in the table path.
- */
- def tableExistsInPath(tablePath: String, conf: Configuration): Boolean = {
- val basePath = new Path(tablePath)
- val fs = basePath.getFileSystem(conf)
- val metaPath = new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)
- fs.exists(metaPath)
- }
-
- 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
- }
- }
/**
* Get the TableIdentifier of the target table in MergeInto.
@@ -256,7 +33,7 @@ object HoodieSqlUtils extends SparkAdapterSupport {
case SubqueryAlias(tableId, _) => tableId
case plan => throw new IllegalArgumentException(s"Illegal plan $plan in target")
}
- sparkAdapter.toTableIdentify(aliaId)
+ sparkAdapter.toTableIdentifier(aliaId)
}
/**
@@ -271,58 +48,4 @@ object HoodieSqlUtils extends SparkAdapterSupport {
case exp => Seq(exp)
}
}
-
- /**
- * Append the spark config and table options to the baseConfig.
- */
- def withSparkConf(spark: SparkSession, options: Map[String, String])
- (baseConfig: Map[String, String] = Map.empty): Map[String, String] = {
- baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority
- (spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options))
- .filterKeys(_.startsWith("hoodie."))
- }
-
- def isEnableHive(sparkSession: SparkSession): Boolean =
- "hive" == sparkSession.sessionState.conf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION)
-
- /**
- * Convert different query instant time format to the commit time format.
- * Currently we support three kinds of instant time format for time travel query:
- * 1、yyyy-MM-dd HH:mm:ss
- * 2、yyyy-MM-dd
- * This will convert to 'yyyyMMdd000000'.
- * 3、yyyyMMddHHmmss
- */
- def formatQueryInstant(queryInstant: String): String = {
- val instantLength = queryInstant.length
- if (instantLength == 19 || instantLength == 23) { // for yyyy-MM-dd HH:mm:ss[.SSS]
- HoodieInstantTimeGenerator.getInstantForDateString(queryInstant)
- } else if (instantLength == HoodieInstantTimeGenerator.SECS_INSTANT_ID_LENGTH
- || instantLength == HoodieInstantTimeGenerator.MILLIS_INSTANT_ID_LENGTH) { // for yyyyMMddHHmmss[SSS]
- HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) // validate the format
- queryInstant
- } else if (instantLength == 10) { // for yyyy-MM-dd
- HoodieActiveTimeline.formatDate(defaultDateFormat.get().parse(queryInstant))
- } else {
- throw new IllegalArgumentException(s"Unsupported query instant time format: $queryInstant,"
- + s"Supported time format are: 'yyyy-MM-dd: HH:mm:ss.SSS' or 'yyyy-MM-dd' or 'yyyyMMddHHmmssSSS'")
- }
- }
-
- def formatName(sparkSession: SparkSession, name: String): String = {
- if (sparkSession.sessionState.conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
- }
-
- /**
- * Check if this is a empty table path.
- */
- def isEmptyPath(tablePath: String, conf: Configuration): Boolean = {
- val basePath = new Path(tablePath)
- val fs = basePath.getFileSystem(conf)
- if (fs.exists(basePath)) {
- fs.listStatus(basePath).isEmpty
- } else {
- true
- }
- }
}
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 31af71994d0bf..ee805838f1b94 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
@@ -17,11 +17,11 @@
package org.apache.spark.sql.hudi.analysis
-import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
import org.apache.hudi.DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL
+import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.HoodieTableMetaClient
-
+import org.apache.hudi.common.util.ReflectionUtils
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, Literal, NamedExpression}
import org.apache.spark.sql.catalyst.plans.Inner
@@ -29,9 +29,10 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation}
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.hudi.HoodieSqlUtils._
import org.apache.spark.sql.hudi.command._
-import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
+import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.{AnalysisException, SparkSession}
@@ -42,12 +43,39 @@ object HoodieAnalysis {
Seq(
session => HoodieResolveReferences(session),
session => HoodieAnalysis(session)
- )
+ ) ++ extraResolutionRules()
def customPostHocResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
Seq(
session => HoodiePostAnalysisRule(session)
- )
+ ) ++ extraPostHocResolutionRules()
+
+ def extraResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] = {
+ if (HoodieSparkUtils.isSpark3_2) {
+ val spark3AnalysisClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3Analysis"
+ val spark3Analysis: SparkSession => Rule[LogicalPlan] =
+ session => ReflectionUtils.loadClass(spark3AnalysisClass, session).asInstanceOf[Rule[LogicalPlan]]
+
+ val spark3ResolveReferences = "org.apache.spark.sql.hudi.analysis.HoodieSpark3ResolveReferences"
+ val spark3References: SparkSession => Rule[LogicalPlan] =
+ session => ReflectionUtils.loadClass(spark3ResolveReferences, session).asInstanceOf[Rule[LogicalPlan]]
+
+ Seq(spark3Analysis, spark3References)
+ } else {
+ Seq.empty
+ }
+ }
+
+ def extraPostHocResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
+ if (HoodieSparkUtils.isSpark3_2) {
+ val spark3PostHocResolutionClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3PostAnalysisRule"
+ val spark3PostHocResolution: SparkSession => Rule[LogicalPlan] =
+ session => ReflectionUtils.loadClass(spark3PostHocResolutionClass, session).asInstanceOf[Rule[LogicalPlan]]
+
+ Seq(spark3PostHocResolution)
+ } else {
+ Seq.empty
+ }
}
/**
@@ -61,7 +89,7 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
plan match {
// Convert to MergeIntoHoodieTableCommand
case m @ MergeIntoTable(target, _, _, _, _)
- if m.resolved && isHoodieTable(target, sparkSession) =>
+ if m.resolved && sparkAdapter.isHoodieTable(target, sparkSession) =>
MergeIntoHoodieTableCommand(m)
// Convert to UpdateHoodieTableCommand
@@ -122,7 +150,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp {
// Resolve merge into
case mergeInto @ MergeIntoTable(target, source, mergeCondition, matchedActions, notMatchedActions)
- if isHoodieTable(target, sparkSession) && target.resolved =>
+ if sparkAdapter.isHoodieTable(target, sparkSession) && target.resolved =>
val resolver = sparkSession.sessionState.conf.resolver
val resolvedSource = analyzer.execute(source)
@@ -161,8 +189,8 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
val resolvedCondition = condition.map(resolveExpressionFrom(resolvedSource)(_))
val resolvedAssignments = if (isInsertOrUpdateStar(assignments)) {
// assignments is empty means insert * or update set *
- val resolvedSourceOutput = resolvedSource.output.filter(attr => !HoodieSqlUtils.isMetaField(attr.name))
- val targetOutput = target.output.filter(attr => !HoodieSqlUtils.isMetaField(attr.name))
+ val resolvedSourceOutput = resolvedSource.output.filter(attr => !HoodieSqlCommonUtils.isMetaField(attr.name))
+ val targetOutput = target.output.filter(attr => !HoodieSqlCommonUtils.isMetaField(attr.name))
val resolvedSourceColumnNames = resolvedSourceOutput.map(_.name)
if(targetOutput.filter(attr => resolvedSourceColumnNames.exists(resolver(_, attr.name))).equals(targetOutput)){
@@ -182,7 +210,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
// For Spark3.2, InsertStarAction/UpdateStarAction's assignments will contain the meta fields.
val withoutMetaAttrs = assignments.filterNot{ assignment =>
if (assignment.key.isInstanceOf[Attribute]) {
- HoodieSqlUtils.isMetaField(assignment.key.asInstanceOf[Attribute].name)
+ HoodieSqlCommonUtils.isMetaField(assignment.key.asInstanceOf[Attribute].name)
} else {
false
}
@@ -303,7 +331,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
val (table, partition, query, overwrite, ifPartitionNotExists) =
sparkAdapter.getInsertIntoChildren(l).get
- if (isHoodieTable(table, sparkSession) && query.resolved &&
+ if (sparkAdapter.isHoodieTable(table, sparkSession) && query.resolved &&
!containUnResolvedStar(query) &&
!checkAlreadyAppendMetaField(query)) {
val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala.map(
@@ -333,7 +361,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
.setBasePath(tablePath)
.setConf(sparkSession.sessionState.newHadoopConf())
.build()
- val tableSchema = HoodieSqlUtils.getTableSqlSchema(metaClient)
+ val tableSchema = HoodieSqlCommonUtils.getTableSqlSchema(metaClient)
if (tableSchema.isDefined && tableDesc.schema.isEmpty) {
// Fill the schema with the schema from the table
c.copy(tableDesc.copy(schema = tableSchema.get))
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala
index be66584778b4e..1363fb939b4e3 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala
@@ -17,21 +17,20 @@
package org.apache.spark.sql.hudi.command
-import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils}
import org.apache.hudi.client.WriteStatus
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.{HoodieTimer, Option => HOption}
import org.apache.hudi.exception.HoodieException
-
+import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils}
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
-import org.apache.spark.sql.catalyst.plans.logical.{CompactionOperation, LogicalPlan}
-import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation
import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE}
-import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import org.apache.spark.sql.types.StringType
+import org.apache.spark.sql.{Row, SparkSession}
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
@@ -50,7 +49,7 @@ case class CompactionHoodiePathCommand(path: String,
val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(
- HoodieSqlUtils.withSparkConf(sparkSession, Map.empty)(
+ HoodieSqlCommonUtils.withSparkConf(sparkSession, Map.empty)(
Map(
DataSourceWriteOptions.TABLE_TYPE.key() -> HoodieTableType.MERGE_ON_READ.name()
)
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala
index 27fb6e779a250..2c89ed8c9d203 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala
@@ -17,13 +17,12 @@
package org.apache.spark.sql.hudi.command
-import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.getTableLocation
import org.apache.spark.sql.types.StringType
+import org.apache.spark.sql.{Row, SparkSession}
case class CompactionHoodieTableCommand(table: CatalogTable,
operation: CompactionOperation, instantTimestamp: Option[Long])
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala
index 7502bf7623aad..a9176164f4c6c 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala
@@ -17,12 +17,11 @@
package org.apache.spark.sql.hudi.command
-import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.getTableLocation
import org.apache.spark.sql.types.{IntegerType, StringType}
+import org.apache.spark.sql.{Row, SparkSession}
case class CompactionShowHoodieTableCommand(table: CatalogTable, limit: Int)
extends HoodieLeafRunnableCommand {
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala
index 2790ea97c87c6..572013981d698 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala
@@ -28,7 +28,8 @@ import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.execution.command.DataWritingCommand
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import scala.collection.JavaConverters._
@@ -71,7 +72,7 @@ case class CreateHoodieTableAsSelectCommand(
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableWithSchema)
val tablePath = hoodieCatalogTable.tableLocation
val hadoopConf = sparkSession.sessionState.newHadoopConf()
- assert(HoodieSqlUtils.isEmptyPath(tablePath, hadoopConf),
+ assert(HoodieSqlCommonUtils.isEmptyPath(tablePath, hadoopConf),
s"Path '$tablePath' should be empty for CTAS")
// Execute the insert query
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala
index a77acf066a76d..2d86626e51d87 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala
@@ -22,11 +22,10 @@ import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.ddl.HiveSyncMode
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
-
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
-import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, LogicalPlan}
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.types.StructType
case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends HoodieLeafRunnableCommand
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala
index aa9d9b8123d8d..954f08ce645c5 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, HoodieCatalogTable}
import org.apache.spark.sql.hive.HiveClientUtils
-import org.apache.spark.sql.hudi.HoodieSqlUtils.isEnableHive
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive
import scala.util.control.NonFatal
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala
index 37d30c813588d..3dd1bc7613591 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala
@@ -17,32 +17,24 @@
package org.apache.spark.sql.hudi.command
-import org.apache.avro.Schema
-import org.apache.avro.generic.{GenericRecord, IndexedRecord}
-
import org.apache.hudi.DataSourceWriteOptions._
-import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, OverwriteWithLatestAvroPayload}
-import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
-import org.apache.hudi.exception.HoodieDuplicateKeyException
import org.apache.hudi.hive.MultiPartKeysValueExtractor
import org.apache.hudi.hive.ddl.HiveSyncMode
import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.sql.InsertMode
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter}
-
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.expressions.{Alias, Literal}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
-import java.util.Properties
-
import scala.collection.JavaConverters._
/**
@@ -290,21 +282,3 @@ object InsertIntoHoodieTableCommand extends Logging {
}
}
}
-
-/**
- * Validate the duplicate key for insert statement without enable the INSERT_DROP_DUPS_OPT
- * config.
- */
-class ValidateDuplicateKeyPayload(record: GenericRecord, orderingVal: Comparable[_])
- extends DefaultHoodieRecordPayload(record, orderingVal) {
-
- def this(record: HOption[GenericRecord]) {
- this(if (record.isPresent) record.get else null, 0)
- }
-
- override def combineAndGetUpdateValue(currentValue: IndexedRecord,
- schema: Schema, properties: Properties): HOption[IndexedRecord] = {
- val key = currentValue.asInstanceOf[GenericRecord].get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
- throw new HoodieDuplicateKeyException(key)
- }
-}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
index 2d36c6c31fb3f..b92479868990a 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
@@ -24,18 +24,18 @@ import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.MultiPartKeysValueExtractor
import org.apache.hudi.hive.ddl.HiveSyncMode
-import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils, SparkAdapterSupport}
-
+import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, SparkAdapterSupport}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, BoundReference, Cast, EqualTo, Expression, Literal}
import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
+import org.apache.spark.sql.hudi.HoodieSqlUtils.getMergeIntoTargetTableId
+import org.apache.spark.sql.hudi.SerDeUtils
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload._
-import org.apache.spark.sql.hudi.SerDeUtils
import org.apache.spark.sql.types.{BooleanType, StructType}
import java.util.Base64
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala
index 0ff7ffb45203a..07442c6c877a8 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala
@@ -24,14 +24,13 @@ import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.MultiPartKeysValueExtractor
import org.apache.hudi.hive.ddl.HiveSyncMode
-
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression}
-import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan, UpdateTable}
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, UpdateTable}
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.types.StructField
import scala.collection.JavaConverters._
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
index 35f4a61fe1bb2..7c32d04ec3481 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala
@@ -33,7 +33,7 @@ import org.apache.hudi.functional.TestBootstrap
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.testutils.DataSourceTestUtils
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.functions.{expr, lit}
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
@@ -96,11 +96,18 @@ class TestHoodieSparkSqlWriter {
* Utility method for initializing the spark context.
*/
def initSparkContext(): Unit = {
+ val sparkConf = new SparkConf()
+ if (HoodieSparkUtils.isSpark3_2) {
+ sparkConf.set("spark.sql.catalog.spark_catalog",
+ "org.apache.spark.sql.hudi.catalog.HoodieCatalog")
+ }
+
spark = SparkSession.builder()
.appName(hoodieFooTableName)
.master("local[2]")
.withExtensions(new HoodieSparkSessionExtension)
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+ .config(sparkConf)
.getOrCreate()
sc = spark.sparkContext
sc.setLogLevel("ERROR")
@@ -770,7 +777,7 @@ class TestHoodieSparkSqlWriter {
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableModifier, df1)
- val snapshotDF1 = spark.read.format("org.apache.hudi")
+ val snapshotDF1 = spark.read.format("hudi")
.load(tempBasePath + "/*/*/*/*")
assertEquals(10, snapshotDF1.count())
// remove metadata columns so that expected and actual DFs can be compared as is
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
index 9996f23c728c3..f420b296e2b3a 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
@@ -17,8 +17,6 @@
package org.apache.hudi.functional
-import java.util.Properties
-
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala
index e2521047f68cf..9a74d23c2f937 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala
@@ -65,7 +65,7 @@ class TestAlterTable extends TestHoodieSqlBase {
spark.sql(s"alter table $newTableName add columns(ext0 string)")
val table = spark.sessionState.catalog.getTableMetadata(new TableIdentifier(newTableName))
assertResult(Seq("id", "name", "price", "ts", "ext0")) {
- HoodieSqlUtils.removeMetaFields(table.schema).fields.map(_.name)
+ HoodieSqlCommonUtils.removeMetaFields(table.schema).fields.map(_.name)
}
checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")(
Seq(1, "a1", 10.0, 1000, null)
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 a5b49cc3683d0..4419099bb3551 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
@@ -18,8 +18,10 @@
package org.apache.spark.sql.hudi
import org.apache.hadoop.fs.Path
+import org.apache.hudi.HoodieSparkUtils
import org.apache.hudi.common.fs.FSUtils
import org.apache.log4j.Level
+import org.apache.spark.SparkConf
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.util.Utils
@@ -49,10 +51,20 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll {
.config("hoodie.delete.shuffle.parallelism", "4")
.config("spark.sql.warehouse.dir", sparkWareHouse.getCanonicalPath)
.config("spark.sql.session.timeZone", "CTT")
+ .config(sparkConf())
.getOrCreate()
private var tableId = 0
+ def sparkConf(): SparkConf = {
+ val sparkConf = new SparkConf()
+ if (HoodieSparkUtils.isSpark3_2) {
+ sparkConf.set("spark.sql.catalog.spark_catalog",
+ "org.apache.spark.sql.hudi.catalog.HoodieCatalog")
+ }
+ sparkConf
+ }
+
protected def withTempDir(f: File => Unit): Unit = {
val tempDir = Utils.createTempDir()
try f(tempDir) finally {
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala
index ee1e2e6f42cff..f3274dbe57ec0 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.hudi
+import org.apache.hudi.HoodieSparkUtils
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.exception.HoodieDuplicateKeyException
@@ -288,6 +289,16 @@ class TestInsertTable extends TestHoodieSqlBase {
}
}
+ def moreColsMessage(): String = {
+ "assertion failed: Required select columns count: 4, Current select columns(including static partition column)" +
+ " count: 5,columns: (1,a1,10,2021-06-20,dt)"
+ }
+
+ def lessColsMessage(): String = {
+ "assertion failed: Required select columns count: 4, Current select columns(including static partition column)" +
+ " count: 3,columns: (1,a1,10)"
+ }
+
test("Test Insert Exception") {
val tableName = generateTableName
spark.sql(
@@ -301,15 +312,9 @@ class TestInsertTable extends TestHoodieSqlBase {
| tblproperties (primaryKey = 'id')
| partitioned by (dt)
""".stripMargin)
- checkException(s"insert into $tableName partition(dt = '2021-06-20')" +
- s" select 1, 'a1', 10, '2021-06-20'") (
- "assertion failed: Required select columns count: 4, Current select columns(including static partition column)" +
- " count: 5,columns: (1,a1,10,2021-06-20,dt)"
- )
- checkException(s"insert into $tableName select 1, 'a1', 10")(
- "assertion failed: Required select columns count: 4, Current select columns(including static partition column)" +
- " count: 3,columns: (1,a1,10)"
- )
+ checkExceptionContain(s"insert into $tableName partition(dt = '2021-06-20')" +
+ s" select 1, 'a1', 10, '2021-06-20'") (moreColsMessage())
+ checkExceptionContain(s"insert into $tableName select 1, 'a1', 10")(lessColsMessage())
spark.sql("set hoodie.sql.bulk.insert.enable = true")
spark.sql("set hoodie.sql.insert.mode = strict")
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala
index baac82f4bd153..28dee88e1f61e 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala
@@ -87,7 +87,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
| on s0.id = $tableName.id
| when matched then update set
| id = s0.id, name = s0.name, price = s0.price + $tableName.price, ts = s0.ts
- | when not matched and id % 2 = 0 then insert *
+ | when not matched and s0.id % 2 = 0 then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, ts from $tableName")(
Seq(1, "a1", 30.0, 1002),
@@ -102,9 +102,9 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
| select 1 as id, 'a1' as name, 12 as price, 1003 as ts
| ) s0
| on s0.id = $tableName.id
- | when matched and id != 1 then update set
+ | when matched and s0.id != 1 then update set
| id = s0.id, name = s0.name, price = s0.price, ts = s0.ts
- | when matched and id = 1 then delete
+ | when matched and s0.id = 1 then delete
| when not matched then insert *
""".stripMargin)
val cnt = spark.sql(s"select * from $tableName where id = 1").count()
@@ -178,7 +178,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
| )
| ) s0
| on s0.s_id = t0.id
- | when matched and ts = 1001 then update set id = s0.s_id, name = t0.name, price =
+ | when matched and s0.ts = 1001 then update set id = s0.s_id, name = t0.name, price =
| s0.price, ts = s0.ts
""".stripMargin
)
@@ -233,7 +233,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
| select 1 as id, 'a1' as name, 12 as price, 1001 as ts, '2021-03-21' as dt
| ) as s0
| on t0.id = s0.id
- | when matched and id % 2 = 0 then update set *
+ | when matched and s0.id % 2 = 0 then update set *
""".stripMargin
)
checkAnswer(s"select id,name,price,dt from $tableName")(
@@ -488,7 +488,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
|merge into $targetTable t0
|using $sourceTable s0
|on t0.id = s0.id
- |when matched and cast(_ts as string) > '1000' then update set *
+ |when matched and cast(s0._ts as string) > '1000' then update set *
""".stripMargin)
checkAnswer(s"select id, name, price, _ts from $targetTable")(
Seq(1, "a1", 12, 1001)
@@ -512,7 +512,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
|using $sourceTable s0
|on t0.id = s0.id
|when matched then update set *
- |when not matched and name = 'a2' then insert *
+ |when not matched and s0.name = 'a2' then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, _ts from $targetTable order by id")(
Seq(1, "a1", 12, 1001),
diff --git a/hudi-spark-datasource/hudi-spark2-common/pom.xml b/hudi-spark-datasource/hudi-spark2-common/pom.xml
new file mode 100644
index 0000000000000..403c2fe1e9db2
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark2-common/pom.xml
@@ -0,0 +1,19 @@
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.11.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark2-common
+
+
+ 8
+ 8
+
+
+
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark2-extensions/pom.xml b/hudi-spark-datasource/hudi-spark2-extensions/pom.xml
new file mode 100644
index 0000000000000..163d3481d2f6e
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark2-extensions/pom.xml
@@ -0,0 +1,401 @@
+
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.11.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark2-extensions_${scala.binary.version}
+
+
+ 8
+ 8
+
+
+
+
+
+ org.apache.hudi
+ hudi-spark_${scala.binary.version}
+ ${project.version}
+
+
+
+
+ org.scala-lang
+ scala-library
+ ${scala.version}
+
+
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-hadoop-mr
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-hive-sync
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-sync-common
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.curator
+ *
+
+
+
+
+
+ org.apache.hudi
+ hudi-spark2_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.hudi
+ *
+
+
+
+
+
+
+ log4j
+ log4j
+
+
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+
+
+ com.fasterxml.jackson.module
+ jackson-module-scala_${scala.binary.version}
+
+
+
+
+ org.apache.avro
+ avro
+
+
+
+ com.thoughtworks.paranamer
+ paranamer
+
+
+
+
+
+
+ org.apache.parquet
+ parquet-avro
+
+
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ ${spark2.version}
+
+
+ javax.servlet
+ *
+
+
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ ${spark2.version}
+
+
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ ${spark2.version}
+ tests
+ test
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ ${spark2.version}
+ tests
+ test
+
+
+ org.apache.spark
+ spark-catalyst_${scala.binary.version}
+ ${spark2.version}
+ tests
+ test
+
+
+
+
+ org.apache.spark
+ spark-avro_${scala.binary.version}
+ ${spark2.version}
+ provided
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+
+
+ javax.servlet
+ *
+
+
+ provided
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ javax.servlet
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ provided
+
+
+
+
+ ${hive.groupid}
+ hive-exec
+ ${hive.version}
+ ${hive.exec.classifier}
+
+
+ javax.mail
+ mail
+
+
+ org.eclipse.jetty.aggregate
+ *
+
+
+
+
+ ${hive.groupid}
+ hive-jdbc
+ ${hive.version}
+
+
+ javax.servlet
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+
+
+ ${hive.groupid}
+ hive-metastore
+ ${hive.version}
+
+
+ javax.servlet
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+
+
+ ${hive.groupid}
+ hive-common
+ ${hive.version}
+
+
+ org.eclipse.jetty.orbit
+ javax.servlet
+
+
+
+
+
+ org.apache.curator
+ curator-framework
+ ${zk-curator.version}
+
+
+
+ org.apache.curator
+ curator-client
+ ${zk-curator.version}
+
+
+
+ org.apache.curator
+ curator-recipes
+ ${zk-curator.version}
+
+
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+
+ org.scalatest
+ scalatest_${scala.binary.version}
+ ${scalatest.version}
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+
+
+
+ org.junit.vintage
+ junit-vintage-engine
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+ org.mockito
+ mockito-junit-jupiter
+ test
+
+
+
+ org.junit.platform
+ junit-platform-runner
+ test
+
+
+
+ org.junit.platform
+ junit-platform-suite-api
+ test
+
+
+
+ org.slf4j
+ slf4j-api
+ ${slf4j.version}
+ test
+
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+ tests
+ test
+
+
+
+ org.mortbay.jetty
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ javax.servlet
+ *
+
+
+
+
+
+
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000000000..673594302f8b3
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1,19 @@
+
+# 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.
+
+
+org.apache.hudi.Spark2DefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/hudi/Spark2DefaultSource.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/hudi/Spark2DefaultSource.scala
new file mode 100644
index 0000000000000..6f42eb777c6c0
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/hudi/Spark2DefaultSource.scala
@@ -0,0 +1,28 @@
+/*
+ * 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.spark.sql.sources._
+
+/**
+ * Hoodie Spark Datasource, for reading and writing hoodie tables
+ *
+ */
+class Spark2DefaultSource extends DefaultSource with DataSourceRegister {
+ override def shortName(): String = "hudi"
+}
diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala
index d47e7fbb497b0..bf1cd24484c1a 100644
--- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.adapter
import org.apache.hudi.Spark2RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
-import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
@@ -31,6 +30,7 @@ import org.apache.spark.sql.execution.datasources.{Spark2ParsePartitionUtil, Spa
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser
import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.{Row, SparkSession}
/**
* The adapter for spark2.
@@ -41,11 +41,11 @@ class Spark2Adapter extends SparkAdapter {
new Spark2RowSerDe(encoder)
}
- override def toTableIdentify(aliasId: AliasIdentifier): TableIdentifier = {
+ override def toTableIdentifier(aliasId: AliasIdentifier): TableIdentifier = {
TableIdentifier(aliasId.identifier, aliasId.database)
}
- override def toTableIdentify(relation: UnresolvedRelation): TableIdentifier = {
+ override def toTableIdentifier(relation: UnresolvedRelation): TableIdentifier = {
relation.tableIdentifier
}
@@ -58,7 +58,7 @@ class Spark2Adapter extends SparkAdapter {
}
override def getInsertIntoChildren(plan: LogicalPlan):
- Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)] = {
+ Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)] = {
plan match {
case InsertIntoTable(table, partition, query, overwrite, ifPartitionNotExists) =>
Some((table, partition, query, overwrite, ifPartitionNotExists))
diff --git a/hudi-spark-datasource/hudi-spark3-common/pom.xml b/hudi-spark-datasource/hudi-spark3-common/pom.xml
new file mode 100644
index 0000000000000..affa987372963
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3-common/pom.xml
@@ -0,0 +1,247 @@
+
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.11.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark3-common
+
+
+ ${project.parent.parent.basedir}
+ 8
+ 8
+
+
+
+
+
+ src/main/resources
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ ${scala-maven-plugin.version}
+
+
+ -nobootcp
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-dependencies
+ prepare-package
+
+ copy-dependencies
+
+
+ ${project.build.directory}/lib
+ true
+ true
+ true
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+
+
+ scala-compile-first
+ process-resources
+
+ add-source
+ compile
+
+
+
+ scala-test-compile
+ process-test-resources
+
+ testCompile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+ compile
+
+ compile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ ${skip.hudi-spark3.unit.tests}
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+ org.scalastyle
+ scalastyle-maven-plugin
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+
+
+
+
+ org.scala-lang
+ scala-library
+ ${scala12.version}
+
+
+
+ org.apache.spark
+ spark-sql_2.12
+ ${spark3.version}
+ true
+
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ ${fasterxml.spark3.version}
+
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+
+
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3RowSerDe.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/hudi/Spark3RowSerDe.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3RowSerDe.scala
rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/hudi/Spark3RowSerDe.scala
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala
similarity index 75%
rename from hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala
rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala
index 7e806f7407b2b..aab6b37708107 100644
--- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala
@@ -20,8 +20,7 @@ package org.apache.spark.sql.adapter
import org.apache.hudi.Spark3RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.spark3.internal.ReflectUtil
-
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
@@ -30,7 +29,9 @@ import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
-import org.apache.spark.sql.execution.datasources.{Spark3ParsePartitionUtil, SparkParsePartitionUtil}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.{LogicalRelation, Spark3ParsePartitionUtil, SparkParsePartitionUtil}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.internal.SQLConf
@@ -43,10 +44,10 @@ class Spark3Adapter extends SparkAdapter {
new Spark3RowSerDe(encoder)
}
- override def toTableIdentify(aliasId: AliasIdentifier): TableIdentifier = {
+ override def toTableIdentifier(aliasId: AliasIdentifier): TableIdentifier = {
aliasId match {
case AliasIdentifier(name, Seq(database)) =>
- TableIdentifier(name, Some(database))
+ TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq(_, database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq()) =>
@@ -55,7 +56,7 @@ class Spark3Adapter extends SparkAdapter {
}
}
- override def toTableIdentify(relation: UnresolvedRelation): TableIdentifier = {
+ override def toTableIdentifier(relation: UnresolvedRelation): TableIdentifier = {
relation.multipartIdentifier.asTableIdentifier
}
@@ -78,7 +79,7 @@ class Spark3Adapter extends SparkAdapter {
}
override def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]],
- query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan = {
+ query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan = {
ReflectUtil.createInsertInto(table, partition, Seq.empty[String], query, overwrite, ifPartitionNotExists)
}
@@ -93,4 +94,18 @@ class Spark3Adapter extends SparkAdapter {
override def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String] = {
parser.parseMultipartIdentifier(sqlText)
}
+
+ override def isHoodieTable(table: LogicalPlan, spark: SparkSession): Boolean = {
+ tripAlias(table) match {
+ case LogicalRelation(_, _, Some(tbl), _) => isHoodieTable(tbl)
+ case relation: UnresolvedRelation =>
+ isHoodieTable(toTableIdentifier(relation), spark)
+ case DataSourceV2Relation(table: Table, _, _, _, _) => isHoodieTable(table.properties())
+ case _=> false
+ }
+ }
+
+ def isHoodieTable(map: java.util.Map[String, String]): Boolean = {
+ map.getOrDefault("provider", "").equals("hudi")
+ }
}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala
similarity index 84%
rename from hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala
rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala
index d993b980367fc..81071602db98c 100644
--- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala
@@ -48,11 +48,11 @@ class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
* And this method will generate and return `InternalRow` directly instead of `PartitionValues`.
*/
override def parsePartition(
- path: Path,
- typeInference: Boolean,
- basePaths: Set[Path],
- userSpecifiedDataTypes: Map[String, DataType],
- timeZone: TimeZone): InternalRow = {
+ path: Path,
+ typeInference: Boolean,
+ basePaths: Set[Path],
+ userSpecifiedDataTypes: Map[String, DataType],
+ timeZone: TimeZone): InternalRow = {
val dateFormatter = ReflectUtil.getDateFormatter(timeZone.toZoneId)
val timestampFormatter = TimestampFormatter(timestampPartitionPattern,
timeZone.toZoneId, isParsing = true)
@@ -85,14 +85,14 @@ class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
}
private def parsePartition(
- path: Path,
- typeInference: Boolean,
- basePaths: Set[Path],
- userSpecifiedDataTypes: Map[String, DataType],
- validatePartitionColumns: Boolean,
- zoneId: ZoneId,
- dateFormatter: DateFormatter,
- timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = {
+ path: Path,
+ typeInference: Boolean,
+ basePaths: Set[Path],
+ userSpecifiedDataTypes: Map[String, DataType],
+ validatePartitionColumns: Boolean,
+ zoneId: ZoneId,
+ dateFormatter: DateFormatter,
+ timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = {
val columns = ArrayBuffer.empty[(String, TypedPartValue)]
// Old Hadoop versions don't have `Path.isRoot`
@@ -130,7 +130,7 @@ class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
// i.e. currentPath.getParent == null. For the example of "/table/a=1/",
// the top level dir is "/table".
finished =
- (maybeColumn.isEmpty && !columns.isEmpty) || currentPath.getParent == null
+ (maybeColumn.isEmpty && !columns.isEmpty) || currentPath.getParent == null
if (!finished) {
// For the above example, currentPath will be "/table/".
@@ -148,13 +148,13 @@ class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
}
private def parsePartitionColumn(
- columnSpec: String,
- typeInference: Boolean,
- userSpecifiedDataTypes: Map[String, DataType],
- validatePartitionColumns: Boolean,
- zoneId: ZoneId,
- dateFormatter: DateFormatter,
- timestampFormatter: TimestampFormatter): Option[(String, TypedPartValue)] = {
+ columnSpec: String,
+ typeInference: Boolean,
+ userSpecifiedDataTypes: Map[String, DataType],
+ validatePartitionColumns: Boolean,
+ zoneId: ZoneId,
+ dateFormatter: DateFormatter,
+ timestampFormatter: TimestampFormatter): Option[(String, TypedPartValue)] = {
val equalSignIndex = columnSpec.indexOf('=')
if (equalSignIndex == -1) {
None
@@ -182,11 +182,11 @@ class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
}
private def inferPartitionColumnValue(
- raw: String,
- typeInference: Boolean,
- zoneId: ZoneId,
- dateFormatter: DateFormatter,
- timestampFormatter: TimestampFormatter): DataType = {
+ raw: String,
+ typeInference: Boolean,
+ zoneId: ZoneId,
+ dateFormatter: DateFormatter,
+ timestampFormatter: TimestampFormatter): DataType = {
val decimalTry = Try {
// `BigDecimal` conversion can fail when the `field` is not a form of number.
val bigDecimal = new JBigDecimal(raw)
@@ -248,9 +248,9 @@ class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil {
}
def castPartValueToDesiredType(
- desiredType: DataType,
- value: String,
- zoneId: ZoneId): Any = desiredType match {
+ desiredType: DataType,
+ value: String,
+ zoneId: ZoneId): Any = desiredType match {
case _ if value == DEFAULT_PARTITION_PATH => null
case NullType => null
case StringType => UTF8String.fromString(unescapePathName(value))
diff --git a/hudi-spark-datasource/hudi-spark3-extensions/pom.xml b/hudi-spark-datasource/hudi-spark3-extensions/pom.xml
new file mode 100644
index 0000000000000..1b9a3d9d622e6
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3-extensions/pom.xml
@@ -0,0 +1,552 @@
+
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.11.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark3-extensions_${scala.binary.version}
+
+
+ ${project.parent.parent.basedir}
+
+
+
+
+ org.scala-lang
+ scala-library
+ ${scala12.version}
+
+
+
+ org.apache.spark
+ spark-avro_${scala.binary.version}
+ provided
+
+
+
+ org.apache.spark
+ spark-sql_2.12
+ ${spark3.version}
+ true
+
+
+ org.apache.spark
+ spark-core_2.12
+ ${spark3.version}
+
+
+ javax.servlet
+ *
+
+
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ tests
+ test
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ tests
+ test
+
+
+
+
+ org.apache.parquet
+ parquet-avro
+
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ ${fasterxml.spark3.version}
+
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark3_2.12
+ ${project.version}
+
+
+
+ io.netty
+ netty
+ 3.9.9.Final
+ true
+
+
+ io.netty
+ netty-all
+ 4.1.17.Final
+ true
+
+
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark_${scala.binary.version}
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+
+ org.scalatest
+ scalatest_${scala.binary.version}
+ ${scalatest.version}
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+
+
+
+ org.junit.vintage
+ junit-vintage-engine
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+ org.mockito
+ mockito-junit-jupiter
+ test
+
+
+
+ org.junit.platform
+ junit-platform-runner
+ test
+
+
+
+ org.junit.platform
+ junit-platform-suite-api
+ test
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+
+
+ javax.servlet
+ *
+
+
+ provided
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ javax.servlet
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ provided
+
+
+
+
+ ${hive.groupid}
+ hive-exec
+ ${hive.version}
+ ${hive.exec.classifier}
+
+
+ javax.mail
+ mail
+
+
+ org.eclipse.jetty.aggregate
+ *
+
+
+
+
+ ${hive.groupid}
+ hive-jdbc
+ ${hive.version}
+
+
+ javax.servlet
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+
+
+ ${hive.groupid}
+ hive-metastore
+ ${hive.version}
+
+
+ javax.servlet
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+
+
+ ${hive.groupid}
+ hive-common
+ ${hive.version}
+
+
+ org.eclipse.jetty.orbit
+ javax.servlet
+
+
+
+
+
+ org.apache.curator
+ curator-framework
+ ${zk-curator.version}
+
+
+
+ org.apache.curator
+ curator-client
+ ${zk-curator.version}
+
+
+
+ org.apache.curator
+ curator-recipes
+ ${zk-curator.version}
+
+
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+
+ org.scalatest
+ scalatest_${scala.binary.version}
+ ${scalatest.version}
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+
+
+
+ org.junit.vintage
+ junit-vintage-engine
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+ org.mockito
+ mockito-junit-jupiter
+ test
+
+
+
+ org.junit.platform
+ junit-platform-runner
+ test
+
+
+
+ org.junit.platform
+ junit-platform-suite-api
+ test
+
+
+
+ org.slf4j
+ slf4j-api
+ ${slf4j.version}
+ test
+
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+ tests
+ test
+
+
+
+ org.mortbay.jetty
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ javax.servlet
+ *
+
+
+
+
+
+
+
+
+ src/main/resources
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ ${scala-maven-plugin.version}
+
+
+ -nobootcp
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-dependencies
+ prepare-package
+
+ copy-dependencies
+
+
+ ${project.build.directory}/lib
+ true
+ true
+ true
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+
+
+ scala-compile-first
+ process-resources
+
+ add-source
+ compile
+
+
+
+ scala-test-compile
+ process-test-resources
+
+ testCompile
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+ compile
+
+ compile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ ${skip.hudi-spark3.unit.tests}
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+ org.scalastyle
+ scalastyle-maven-plugin
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+
+
+
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
new file mode 100644
index 0000000000000..4d4174781cb94
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
@@ -0,0 +1,250 @@
+
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.11.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark3.1.x_2.12
+
+
+ ${project.parent.parent.basedir}
+ 8
+ 8
+
+
+
+
+
+ src/main/resources
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ ${scala-maven-plugin.version}
+
+
+ -nobootcp
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-dependencies
+ prepare-package
+
+ copy-dependencies
+
+
+ ${project.build.directory}/lib
+ true
+ true
+ true
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+
+
+ scala-compile-first
+ process-resources
+
+ add-source
+ compile
+
+
+
+ scala-test-compile
+ process-test-resources
+
+ testCompile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+ compile
+
+ compile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ ${skip.hudi-spark3.unit.tests}
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+ org.scalastyle
+ scalastyle-maven-plugin
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+
+
+
+
+ org.scala-lang
+ scala-library
+ ${scala12.version}
+
+
+
+ org.apache.spark
+ spark-sql_2.12
+ ${spark3.version}
+ true
+
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ ${fasterxml.spark3.version}
+
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.hudi
+ hudi-spark3-common
+ ${project.version}
+
+
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000000000..8fbcd663c623d
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1,19 @@
+
+# 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.
+
+
+org.apache.hudi.Spark3xDefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark3xDefaultSource.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark3xDefaultSource.scala
new file mode 100644
index 0000000000000..6f941dabcb9cb
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark3xDefaultSource.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.spark.sql.sources.DataSourceRegister
+
+class Spark3xDefaultSource extends DefaultSource with DataSourceRegister {
+ override def shortName(): String = "hudi"
+}
diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml
index ca09d8359f96b..406f92eb8189e 100644
--- a/hudi-spark-datasource/hudi-spark3/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3/pom.xml
@@ -29,6 +29,8 @@
${project.parent.parent.basedir}
+ 8
+ 8
@@ -187,6 +189,11 @@
hudi-spark-common_${scala.binary.version}
${project.version}
+
+ org.apache.hudi
+ hudi-spark3-common
+ ${project.version}
+
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark3/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000000000..33ab03f55477b
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1,19 @@
+
+# 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.
+
+
+org.apache.hudi.Spark3DefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala
new file mode 100644
index 0000000000000..1ac53d47f5536
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.exception.HoodieException
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.hudi.catalog.HoodieInternalV2Table
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class Spark3DefaultSource extends DefaultSource with DataSourceRegister with TableProvider {
+
+ override def shortName(): String = "hudi"
+
+ def inferSchema: StructType = new StructType() // empty
+
+ override def inferSchema(options: CaseInsensitiveStringMap): StructType = inferSchema
+
+ override def getTable(schema: StructType,
+ partitioning: Array[Transform],
+ properties: java.util.Map[String, String]): Table = {
+ val options = new CaseInsensitiveStringMap(properties)
+ val path = options.get("path")
+ if (path == null) throw new HoodieException("path is null")
+
+ HoodieInternalV2Table(SparkSession.active, path)
+ }
+
+ /*override def supportsExternalMetadata() = true
+
+ override def inferPartitioning(options: CaseInsensitiveStringMap): Array[Transform] = {
+ getTable(null, null, options).partitioning
+ }*/
+
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifierHelper.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifierHelper.scala
new file mode 100644
index 0000000000000..97ba3eb84fd80
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifierHelper.scala
@@ -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.spark.sql.connector.catalog
+
+import java.util
+import java.util.Objects
+
+object HoodieIdentifierHelper {
+ def of(namespace: Array[String], name: String): Identifier = {
+ HoodieIdentifier(namespace, name)
+ }
+}
+
+/**
+ * This class is to make scala-2.11 compilable.
+ * Using Identifier.of(namespace, name) to get a IdentifierImpl will throw
+ * compile exception( Static methods in interface require -target:jvm-1.8)
+ */
+case class HoodieIdentifier(namespace: Array[String], name: String) extends Identifier {
+
+ override def equals(o: Any): Boolean = {
+ o match {
+ case that: HoodieIdentifier => util.Arrays.equals(namespace.asInstanceOf[Array[Object]],
+ that.namespace.asInstanceOf[Array[Object]]) && name == that.name
+ case _ => false
+ }
+ }
+
+ override def hashCode: Int = {
+ val nh = namespace.toSeq.hashCode().asInstanceOf[Object]
+ Objects.hash(nh, name)
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableWithV1Fallback.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableWithV1Fallback.scala
new file mode 100644
index 0000000000000..1c8f7acb6196f
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/connector/catalog/V2TableWithV1Fallback.scala
@@ -0,0 +1,27 @@
+/*
+ * 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.connector.catalog
+
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+
+/**
+ * This is copied from spark 3.2.0 code since Spark 3.0.0 do not have the class.
+ */
+private[sql] trait V2TableWithV1Fallback extends Table {
+ def v1Table: CatalogTable
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/SparkSqlUtils.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/SparkSqlUtils.scala
new file mode 100644
index 0000000000000..419a692a63315
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/SparkSqlUtils.scala
@@ -0,0 +1,107 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.exception.HoodieException
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.BucketSpec
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform}
+import org.apache.spark.sql.types.{StringType, StructField}
+import org.apache.spark.sql.{SparkSession, types}
+
+import java.net.URI
+import java.util.Locale
+import scala.collection.JavaConverters.asScalaBufferConverter
+import scala.collection.mutable
+
+object SparkSqlUtils {
+
+ private lazy val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala.toSet
+
+ def isMetaField(name: String): Boolean = {
+ metaFields.contains(name)
+ }
+
+ def removeMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = {
+ attrs.filterNot(attr => isMetaField(attr.name))
+ }
+
+ def addMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = {
+ val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala
+ // filter the meta field to avoid duplicate field.
+ val dataFields = attrs.filterNot(f => metaFields.contains(f.name))
+ metaFields.map(AttributeReference(_, StringType)()) ++ dataFields
+ }
+
+ def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = {
+ val identityCols = new mutable.ArrayBuffer[String]
+ var bucketSpec = Option.empty[BucketSpec]
+
+ partitions.map {
+ case IdentityTransform(FieldReference(Seq(col))) =>
+ identityCols += col
+
+
+ case BucketTransform(numBuckets, FieldReference(Seq(col))) =>
+ bucketSpec = Some(BucketSpec(numBuckets, col :: Nil, Nil))
+
+ case transform =>
+ throw new HoodieException(s"Partitioning by expressions")
+ }
+
+ (identityCols, bucketSpec)
+ }
+
+ def isHoodieTable(properties: Map[String, String]): Boolean = {
+ properties.getOrElse("provider", "").toLowerCase(Locale.ROOT) == "hudi"
+ }
+
+ 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)
+ }
+ val conf = sparkSession.sessionState.newHadoopConf()
+ uri.map(makePathQualified(_, conf))
+ .map(removePlaceHolder)
+ .getOrElse(throw new IllegalArgumentException(s"Missing location for $identifier"))
+ }
+
+ def makePathQualified(path: URI, hadoopConf: Configuration): String = {
+ val hadoopPath = new Path(path)
+ val fs = hadoopPath.getFileSystem(hadoopConf)
+ fs.makeQualified(hadoopPath).toUri.toString
+ }
+
+ private def removePlaceHolder(path: String): String = {
+ if (path == null || path.isEmpty) {
+ path
+ } else if (path.endsWith("-__PLACEHOLDER__")) {
+ path.substring(0, path.length() - 16)
+ } else {
+ path
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
new file mode 100644
index 0000000000000..3b998f3fb99c3
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
@@ -0,0 +1,187 @@
+/*
+ * 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.analysis
+
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.{DefaultSource, SparkAdapterSupport}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{ResolvedTable, UnresolvedPartitionSpec}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, V2SessionCatalog}
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{castIfNeeded, removeMetaFields, tableExistsInPath}
+import org.apache.spark.sql.hudi.catalog.{HoodieCatalog, HoodieConfigHelper, HoodieInternalV2Table}
+import org.apache.spark.sql.hudi.command.{AlterHoodieTableDropPartitionCommand, ShowHoodieTablePartitionsCommand, TruncateHoodieTableCommand}
+import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, SparkSqlUtils}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{AnalysisException, SQLContext, SparkSession}
+
+/**
+ * Rule for convert the logical plan to command.
+ * @param sparkSession
+ */
+case class HoodieSpark3Analysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
+ with SparkAdapterSupport with HoodieConfigHelper {
+
+ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
+ case dsv2 @ DataSourceV2Relation(d: HoodieInternalV2Table, _, _, _, _) =>
+ val output = dsv2.output
+ val catalogTable = if (d.catalogTable.isDefined) {
+ Some(d.v1Table)
+ } else {
+ None
+ }
+ val relation = new DefaultSource().createRelation(new SQLContext(sparkSession),
+ buildHoodieConfig(d.hoodieCatalogTable))
+ LogicalRelation(relation, output, catalogTable, isStreaming = false)
+ case a @ InsertIntoStatement(r: DataSourceV2Relation, partitionSpec, _, _, _, _) if a.query.resolved &&
+ r.table.isInstanceOf[HoodieInternalV2Table] &&
+ needsSchemaAdjustment(a.query, r.table.asInstanceOf[HoodieInternalV2Table], partitionSpec, r.schema) =>
+ val projection = resolveQueryColumnsByOrdinal(a.query, r.output)
+ if (projection != a.query) {
+ a.copy(query = projection)
+ } else {
+ a
+ }
+ }
+
+ private def needsSchemaAdjustment(query: LogicalPlan,
+ hoodieTable: HoodieInternalV2Table,
+ partitionSpec: Map[String, Option[String]],
+ schema: StructType): Boolean = {
+ val output = query.output
+ val queryOutputWithoutMetaFields = removeMetaFields(output)
+ val partitionFields = hoodieTable.hoodieCatalogTable.partitionFields
+ val partitionSchema = hoodieTable.hoodieCatalogTable.partitionSchema
+ val staticPartitionValues = partitionSpec.filter(p => p._2.isDefined).mapValues(_.get)
+
+ assert(staticPartitionValues.isEmpty ||
+ staticPartitionValues.size == partitionSchema.size,
+ s"Required partition columns is: ${partitionSchema.json}, Current static partitions " +
+ s"is: ${staticPartitionValues.mkString("," + "")}")
+
+ assert(staticPartitionValues.size + queryOutputWithoutMetaFields.size
+ == hoodieTable.hoodieCatalogTable.tableSchemaWithoutMetaFields.size,
+ s"Required select columns count: ${hoodieTable.hoodieCatalogTable.tableSchemaWithoutMetaFields.size}, " +
+ s"Current select columns(including static partition column) count: " +
+ s"${staticPartitionValues.size + queryOutputWithoutMetaFields.size},columns: " +
+ s"(${(queryOutputWithoutMetaFields.map(_.name) ++ staticPartitionValues.keys).mkString(",")})")
+
+ // static partition insert.
+ if (staticPartitionValues.nonEmpty) {
+ // drop partition fields in origin schema to align fields.
+ schema.dropWhile(p => partitionFields.contains(p.name))
+ }
+
+ val existingSchemaOutput = output.take(schema.length)
+ existingSchemaOutput.map(_.name) != schema.map(_.name) ||
+ existingSchemaOutput.map(_.dataType) != schema.map(_.dataType)
+ }
+
+ private def resolveQueryColumnsByOrdinal(query: LogicalPlan,
+ targetAttrs: Seq[Attribute]): LogicalPlan = {
+ // always add a Cast. it will be removed in the optimizer if it is unnecessary.
+ val project = query.output.zipWithIndex.map { case (attr, i) =>
+ if (i < targetAttrs.length) {
+ val targetAttr = targetAttrs(i)
+ val castAttr = castIfNeeded(attr.withNullability(targetAttr.nullable), targetAttr.dataType, conf)
+ Alias(castAttr, targetAttr.name)()
+ } else {
+ attr
+ }
+ }
+ Project(project, query)
+ }
+}
+
+/**
+ * Rule for resolve hoodie's extended syntax or rewrite some logical plan.
+ * @param sparkSession
+ */
+case class HoodieSpark3ResolveReferences(sparkSession: SparkSession) extends Rule[LogicalPlan]
+ with SparkAdapterSupport with HoodieConfigHelper {
+
+ def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp {
+ // Fill schema for Create Table without specify schema info
+ case c @ CreateV2Table(tableCatalog, tableName, schema, _, properties, _)
+ if SparkSqlUtils.isHoodieTable(properties) =>
+
+ val hoodieCatalog = tableCatalog match {
+ case catalog: HoodieCatalog => catalog
+ case _ => tableCatalog.asInstanceOf[V2SessionCatalog]
+ }
+ val tablePath = SparkSqlUtils.getTableLocation(properties,
+ TableIdentifier(tableName.name(), tableName.namespace().lastOption), sparkSession)
+
+ val tableExistInCatalog = hoodieCatalog.tableExists(tableName)
+ // Only when the table has not exist in catalog, we need to fill the schema info for creating table.
+ if (!tableExistInCatalog && tableExistsInPath(tablePath, sparkSession.sessionState.newHadoopConf())) {
+ val metaClient = HoodieTableMetaClient.builder()
+ .setBasePath(tablePath)
+ .setConf(sparkSession.sessionState.newHadoopConf())
+ .build()
+ val tableSchema = HoodieSqlCommonUtils.getTableSqlSchema(metaClient)
+ if (tableSchema.isDefined && schema.isEmpty) {
+ // Fill the schema with the schema from the table
+ c.copy(tableSchema = tableSchema.get)
+ } else if (tableSchema.isDefined && schema != tableSchema.get) {
+ throw new AnalysisException(s"Specified schema in create table statement is not equal to the table schema." +
+ s"You should not specify the schema for an exist table: $tableName ")
+ } else {
+ c
+ }
+ } else {
+ c
+ }
+ case DropPartitions(child, specs, ifExists, purge)
+ if child.resolved && child.isInstanceOf[ResolvedTable] && child.asInstanceOf[ResolvedTable].table.isInstanceOf[HoodieInternalV2Table] =>
+ AlterHoodieTableDropPartitionCommand(
+ child.asInstanceOf[ResolvedTable].identifier.asTableIdentifier,
+ specs.seq.map(f => f.asInstanceOf[UnresolvedPartitionSpec]).map(s => s.spec),
+ ifExists,
+ purge,
+ retainData = true
+ )
+ case p => p
+ }
+}
+
+/**
+ * Rule for rewrite some spark commands to hudi's implementation.
+ * @param sparkSession
+ */
+case class HoodieSpark3PostAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {
+ override def apply(plan: LogicalPlan): LogicalPlan = {
+ plan match {
+ case ShowPartitions(child, specOpt, _)
+ if child.isInstanceOf[ResolvedTable] &&
+ child.asInstanceOf[ResolvedTable].table.isInstanceOf[HoodieInternalV2Table] =>
+ ShowHoodieTablePartitionsCommand(child.asInstanceOf[ResolvedTable].identifier.asTableIdentifier, specOpt.map(s => s.asInstanceOf[UnresolvedPartitionSpec].spec))
+
+ // Rewrite TruncateTableCommand to TruncateHoodieTableCommand
+ case TruncateTable(child)
+ if child.isInstanceOf[ResolvedTable] &&
+ child.asInstanceOf[ResolvedTable].table.isInstanceOf[HoodieInternalV2Table] =>
+ new TruncateHoodieTableCommand(child.asInstanceOf[ResolvedTable].identifier.asTableIdentifier, None)
+ case _ => plan
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/BaseStagedTable.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/BaseStagedTable.scala
new file mode 100644
index 0000000000000..c3cbcb80565e1
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/BaseStagedTable.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.catalog
+
+import org.apache.hudi.exception.HoodieException
+import org.apache.spark.sql.connector.catalog._
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+
+case class BaseStagedTable(ident: Identifier,
+ table: Table,
+ catalog: TableCatalog) extends SupportsWrite with StagedTable {
+ override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
+ info match {
+ case supportsWrite: SupportsWrite => supportsWrite.newWriteBuilder(info)
+ case _ => throw new HoodieException(s"Table implementation does not support writes: $ident.name")
+ }
+ }
+
+ override def abortStagedChanges(): Unit = catalog.dropTable(ident)
+
+ override def commitStagedChanges(): Unit = {}
+
+ override def name(): String = ident.name()
+
+ override def schema(): StructType = table.schema()
+
+ override def partitioning(): Array[Transform] = table.partitioning()
+
+ override def capabilities(): util.Set[TableCapability] = table.capabilities()
+
+ override def properties(): util.Map[String, String] = table.properties()
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
new file mode 100644
index 0000000000000..f17a043607f7d
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
@@ -0,0 +1,289 @@
+/*
+ * 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.catalog
+
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.DataSourceWriteOptions
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hive.util.ConfigUtils
+import org.apache.hudi.sql.InsertMode
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils, HoodieCatalogTable}
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange, UpdateColumnType}
+import org.apache.spark.sql.connector.catalog._
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.DataSource
+import org.apache.spark.sql.hudi.command.{AlterHoodieTableAddColumnsCommand, AlterHoodieTableChangeColumnCommand, AlterHoodieTableRenameCommand, CreateHoodieTableCommand}
+import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, SparkSqlUtils}
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.{Dataset, SaveMode, SparkSession, _}
+
+import java.util
+import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter}
+
+class HoodieCatalog extends DelegatingCatalogExtension with StagingTableCatalog with HoodieConfigHelper {
+
+ val spark: SparkSession = SparkSession.active
+
+ override def stageCreate(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = {
+ if (SparkSqlUtils.isHoodieTable(properties.asScala.toMap)) {
+ HoodieStagedTable(ident, this, schema, partitions, properties, TableCreationMode.STAGE_CREATE)
+ } else {
+ BaseStagedTable(
+ ident,
+ super.createTable(ident, schema, partitions, properties),
+ this)
+ }
+ }
+
+ override def stageReplace(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = {
+ if (SparkSqlUtils.isHoodieTable(properties.asScala.toMap)) {
+ HoodieStagedTable(ident, this, schema, partitions, properties, TableCreationMode.STAGE_REPLACE)
+ } else {
+ super.dropTable(ident)
+ BaseStagedTable(
+ ident,
+ super.createTable(ident, schema, partitions, properties),
+ this)
+ }
+ }
+
+ override def stageCreateOrReplace(ident: Identifier,
+ schema: StructType,
+ partitions: Array[Transform],
+ properties: util.Map[String, String]): StagedTable = {
+ if (SparkSqlUtils.isHoodieTable(properties.asScala.toMap)) {
+ HoodieStagedTable(
+ ident, this, schema, partitions, properties, TableCreationMode.CREATE_OR_REPLACE)
+ } else {
+ try super.dropTable(ident) catch {
+ case _: NoSuchTableException => // ignore the exception
+ }
+ BaseStagedTable(
+ ident,
+ super.createTable(ident, schema, partitions, properties),
+ this)
+ }
+ }
+
+ override def loadTable(ident: Identifier): Table = {
+ try {
+ super.loadTable(ident) match {
+ case v1: V1Table if HoodieSqlCommonUtils.isHoodieTable(v1.catalogTable) =>
+ HoodieInternalV2Table(
+ spark,
+ v1.catalogTable.location.toString,
+ catalogTable = Some(v1.catalogTable),
+ tableIdentifier = Some(ident.toString))
+ case o => o
+ }
+ } catch {
+ case e: Exception =>
+ throw e
+ }
+ }
+
+ override def createTable(ident: Identifier,
+ schema: StructType,
+ partitions: Array[Transform],
+ properties: util.Map[String, String]): Table = {
+ createHoodieTable(ident, schema, partitions, properties, Map.empty, Option.empty, TableCreationMode.CREATE)
+ }
+
+ override def tableExists(ident: Identifier): Boolean = super.tableExists(ident)
+
+ override def dropTable(ident: Identifier): Boolean = super.dropTable(ident)
+
+ override def purgeTable(ident: Identifier): Boolean = {
+ val table = loadTable(ident)
+ table match {
+ case hoodieTable: HoodieInternalV2Table =>
+ val location = hoodieTable.hoodieCatalogTable.tableLocation
+ val targetPath = new Path(location)
+ val engineContext = new HoodieSparkEngineContext(spark.sparkContext)
+ val fs = FSUtils.getFs(location, spark.sparkContext.hadoopConfiguration)
+ FSUtils.deleteDir(engineContext, fs, targetPath, spark.sparkContext.defaultParallelism)
+ super.dropTable(ident)
+ case _ =>
+ }
+ true
+ }
+
+ @throws[NoSuchTableException]
+ @throws[TableAlreadyExistsException]
+ override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = {
+ loadTable(oldIdent) match {
+ case _: HoodieInternalV2Table =>
+ new AlterHoodieTableRenameCommand(oldIdent.asTableIdentifier, newIdent.asTableIdentifier, false).run(spark)
+ case _ => super.renameTable(oldIdent, newIdent)
+ }
+ }
+
+ override def alterTable(ident: Identifier, changes: TableChange*): Table = {
+ val tableIdent = TableIdentifier(ident.name(), ident.namespace().lastOption)
+ // scalastyle:off
+ val table = loadTable(ident) match {
+ case hoodieTable: HoodieInternalV2Table => hoodieTable
+ case _ => return super.alterTable(ident, changes: _*)
+ }
+ // scalastyle:on
+
+ val grouped = changes.groupBy(c => c.getClass)
+
+ grouped.foreach {
+ case (t, newColumns) if t == classOf[AddColumn] =>
+ AlterHoodieTableAddColumnsCommand(
+ tableIdent,
+ newColumns.asInstanceOf[Seq[AddColumn]].map { col =>
+ StructField(
+ col.fieldNames()(0),
+ col.dataType(),
+ col.isNullable)
+ }).run(spark)
+ case (t, columnChanges) if classOf[ColumnChange].isAssignableFrom(t) =>
+ columnChanges.foreach {
+ case dataType: UpdateColumnType =>
+ val fieldName = dataType.fieldNames()(0)
+ val newDataType = dataType.newDataType()
+ val structField = StructField(fieldName, newDataType)
+ AlterHoodieTableChangeColumnCommand(tableIdent, fieldName, structField).run(spark)
+ }
+ case (t, _) =>
+ throw new UnsupportedOperationException(s"not supported table change: ${t.getClass}")
+ }
+
+ loadTable(ident)
+ }
+
+ def createHoodieTable(ident: Identifier,
+ schema: StructType,
+ partitions: Array[Transform],
+ allTableProperties: util.Map[String, String],
+ writeOptions: Map[String, String],
+ sourceQuery: Option[DataFrame],
+ operation: TableCreationMode): Table = {
+
+ val (partitionColumns, maybeBucketSpec) = SparkSqlUtils.convertTransforms(partitions)
+ val newSchema = schema
+ val newPartitionColumns = partitionColumns
+ val newBucketSpec = maybeBucketSpec
+
+ val isByPath = isPathIdentifier(ident)
+
+ val location = if (isByPath) Option(ident.name()) else Option(allTableProperties.get("location"))
+ val id = ident.asTableIdentifier
+
+ val locUriOpt = location.map(CatalogUtils.stringToURI)
+ val existingTableOpt = getExistingTableIfExists(id)
+ val loc = locUriOpt
+ .orElse(existingTableOpt.flatMap(_.storage.locationUri))
+ .getOrElse(spark.sessionState.catalog.defaultTablePath(id))
+ val storage = DataSource.buildStorageFormatFromOptions(writeOptions)
+ .copy(locationUri = Option(loc))
+ val tableType =
+ if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED
+ val commentOpt = Option(allTableProperties.get("comment"))
+
+ val tablePropertiesNew = new util.HashMap[String, String](allTableProperties)
+ // put path to table properties.
+ tablePropertiesNew.put("path", loc.getPath)
+
+ val tableDesc = new CatalogTable(
+ identifier = id,
+ tableType = tableType,
+ storage = storage,
+ schema = newSchema,
+ provider = Option("hudi"),
+ partitionColumnNames = newPartitionColumns,
+ bucketSpec = newBucketSpec,
+ properties = tablePropertiesNew.asScala.toMap,
+ comment = commentOpt)
+
+ val hoodieCatalogTable = HoodieCatalogTable(spark, tableDesc)
+
+ if (operation == TableCreationMode.STAGE_CREATE) {
+ val tablePath = hoodieCatalogTable.tableLocation
+ val hadoopConf = spark.sessionState.newHadoopConf()
+ assert(HoodieSqlCommonUtils.isEmptyPath(tablePath, hadoopConf),
+ s"Path '$tablePath' should be empty for CTAS")
+ hoodieCatalogTable.initHoodieTable()
+
+ val tblProperties = hoodieCatalogTable.catalogProperties
+ val options = Map(
+ DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> (tableDesc.tableType == CatalogTableType.MANAGED).toString,
+ DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
+ DataSourceWriteOptions.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(tableDesc.properties.asJava),
+ DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(),
+ DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true"
+ )
+ saveSourceDF(sourceQuery, tableDesc.properties ++ buildHoodieInsertConfig(hoodieCatalogTable, spark, isOverwrite = false, Map.empty, options))
+ CreateHoodieTableCommand.createTableInCatalog(spark, hoodieCatalogTable, ignoreIfExists = false)
+ } else if (sourceQuery.isEmpty) {
+ saveSourceDF(sourceQuery, tableDesc.properties)
+ new CreateHoodieTableCommand(tableDesc, false).run(spark)
+ } else {
+ saveSourceDF(sourceQuery, tableDesc.properties ++ buildHoodieInsertConfig(hoodieCatalogTable, spark, isOverwrite = false, Map.empty, Map.empty))
+ new CreateHoodieTableCommand(tableDesc, false).run(spark)
+ }
+
+ loadTable(ident)
+ }
+
+ private def isPathIdentifier(ident: Identifier) = new Path(ident.name()).isAbsolute
+
+ protected def isPathIdentifier(table: CatalogTable): Boolean = {
+ isPathIdentifier(table.identifier)
+ }
+
+ protected def isPathIdentifier(tableIdentifier: TableIdentifier): Boolean = {
+ isPathIdentifier(HoodieIdentifierHelper.of(tableIdentifier.database.toArray, tableIdentifier.table))
+ }
+
+ private def getExistingTableIfExists(table: TableIdentifier): Option[CatalogTable] = {
+ // If this is a path identifier, we cannot return an existing CatalogTable. The Create command
+ // will check the file system itself
+ val catalog = spark.sessionState.catalog
+ // scalastyle:off
+ if (isPathIdentifier(table)) return None
+ // scalastyle:on
+ val tableExists = catalog.tableExists(table)
+ if (tableExists) {
+ val oldTable = catalog.getTableMetadata(table)
+ if (oldTable.tableType == CatalogTableType.VIEW) throw new HoodieException(
+ s"$table is a view. You may not write data into a view.")
+ if (!HoodieSqlCommonUtils.isHoodieTable(oldTable)) throw new HoodieException(s"$table is not a Hoodie table.")
+ Some(oldTable)
+ } else None
+ }
+
+ private def saveSourceDF(sourceQuery: Option[Dataset[_]],
+ properties: Map[String, String]): Unit = {
+ sourceQuery.map(df => {
+ df.write.format("org.apache.hudi")
+ .options(properties)
+ .mode(SaveMode.Append)
+ .save()
+ df
+ })
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieConfigHelper.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieConfigHelper.scala
new file mode 100644
index 0000000000000..797c794936d98
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieConfigHelper.scala
@@ -0,0 +1,190 @@
+/*
+ * 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.catalog
+
+import org.apache.hudi.DataSourceWriteOptions
+import org.apache.hudi.DataSourceWriteOptions._
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
+import org.apache.hudi.hive.MultiPartKeysValueExtractor
+import org.apache.hudi.hive.ddl.HiveSyncMode
+import org.apache.hudi.keygen.ComplexKeyGenerator
+import org.apache.hudi.sql.InsertMode
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{castIfNeeded, isEnableHive, withSparkConf}
+import org.apache.spark.sql.hudi.command.{SqlKeyGenerator, ValidateDuplicateKeyPayload}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructField
+
+import scala.collection.JavaConverters.propertiesAsScalaMapConverter
+
+trait HoodieConfigHelper extends Logging {
+
+ def buildHoodieConfig(hoodieCatalogTable: HoodieCatalogTable): Map[String, String] = {
+ val sparkSession: SparkSession = hoodieCatalogTable.spark
+ val catalogProperties = hoodieCatalogTable.catalogProperties
+ val tableConfig = hoodieCatalogTable.tableConfig
+ val tableId = hoodieCatalogTable.table.identifier
+
+ val preCombineColumn = Option(tableConfig.getPreCombineField).getOrElse("")
+ assert(hoodieCatalogTable.primaryKeys.nonEmpty,
+ s"There are no primary key in table ${hoodieCatalogTable.table.identifier}, cannot execute update operator")
+ val enableHive = isEnableHive(sparkSession)
+
+ withSparkConf(sparkSession, catalogProperties) {
+ Map(
+ "path" -> hoodieCatalogTable.tableLocation,
+ RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
+ PRECOMBINE_FIELD.key -> preCombineColumn,
+ TBL_NAME.key -> hoodieCatalogTable.tableName,
+ HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable,
+ URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning,
+ KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
+ SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName,
+ OPERATION.key -> UPSERT_OPERATION_OPT_VAL,
+ PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp,
+ META_SYNC_ENABLED.key -> enableHive.toString,
+ HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
+ HIVE_USE_JDBC.key -> "false",
+ HIVE_DATABASE.key -> tableId.database.getOrElse("default"),
+ HIVE_TABLE.key -> tableId.table,
+ HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp,
+ HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
+ HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
+ HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
+ SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL
+ )
+ }
+ }
+
+ def cast(exp: Expression, field: StructField, sqlConf: SQLConf): Expression = {
+ castIfNeeded(exp, field.dataType, sqlConf)
+ }
+
+ /**
+ * Build the default config for insert.
+ * @return
+ */
+ def buildHoodieInsertConfig(hoodieCatalogTable: HoodieCatalogTable,
+ sparkSession: SparkSession,
+ isOverwrite: Boolean,
+ insertPartitions: Map[String, Option[String]] = Map.empty,
+ extraOptions: Map[String, String]): Map[String, String] = {
+
+ if (insertPartitions.nonEmpty &&
+ (insertPartitions.keys.toSet != hoodieCatalogTable.partitionFields.toSet)) {
+ throw new IllegalArgumentException(s"Insert partition fields" +
+ s"[${insertPartitions.keys.mkString(" " )}]" +
+ s" not equal to the defined partition in table[${hoodieCatalogTable.partitionFields.mkString(",")}]")
+ }
+ val path = hoodieCatalogTable.tableLocation
+ val tableType = hoodieCatalogTable.tableTypeName
+ val tableConfig = hoodieCatalogTable.tableConfig
+ val tableSchema = hoodieCatalogTable.tableSchema
+
+ val options = hoodieCatalogTable.catalogProperties ++ tableConfig.getProps.asScala.toMap ++ extraOptions
+ val parameters = withSparkConf(sparkSession, options)()
+
+ val preCombineColumn = hoodieCatalogTable.preCombineKey.getOrElse("")
+ val partitionFields = hoodieCatalogTable.partitionFields.mkString(",")
+
+ val hiveStylePartitioningEnable = Option(tableConfig.getHiveStylePartitioningEnable).getOrElse("true")
+ val urlEncodePartitioning = Option(tableConfig.getUrlEncodePartitioning).getOrElse("false")
+ val keyGeneratorClassName = Option(tableConfig.getKeyGeneratorClassName)
+ .getOrElse(classOf[ComplexKeyGenerator].getCanonicalName)
+
+ val enableBulkInsert = parameters.getOrElse(DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key,
+ DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean
+ val dropDuplicate = sparkSession.conf
+ .getOption(INSERT_DROP_DUPS.key).getOrElse(INSERT_DROP_DUPS.defaultValue).toBoolean
+
+ val insertMode = InsertMode.of(parameters.getOrElse(DataSourceWriteOptions.SQL_INSERT_MODE.key,
+ DataSourceWriteOptions.SQL_INSERT_MODE.defaultValue()))
+ val isNonStrictMode = insertMode == InsertMode.NON_STRICT
+ val isPartitionedTable = hoodieCatalogTable.partitionFields.nonEmpty
+ val hasPrecombineColumn = preCombineColumn.nonEmpty
+ val operation =
+ (enableBulkInsert, isOverwrite, dropDuplicate, isNonStrictMode, isPartitionedTable) match {
+ case (true, _, _, false, _) =>
+ throw new IllegalArgumentException(s"Table with primaryKey can not use bulk insert in ${insertMode.value()} mode.")
+ case (true, true, _, _, true) =>
+ throw new IllegalArgumentException(s"Insert Overwrite Partition can not use bulk insert.")
+ case (true, _, true, _, _) =>
+ throw new IllegalArgumentException(s"Bulk insert cannot support drop duplication." +
+ s" Please disable $INSERT_DROP_DUPS and try again.")
+ // if enableBulkInsert is true, use bulk insert for the insert overwrite non-partitioned table.
+ case (true, true, _, _, false) => BULK_INSERT_OPERATION_OPT_VAL
+ // insert overwrite table
+ case (false, true, _, _, false) => INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL
+ // insert overwrite partition
+ case (_, true, _, _, true) => INSERT_OVERWRITE_OPERATION_OPT_VAL
+ // disable dropDuplicate, and provide preCombineKey, use the upsert operation for strict and upsert mode.
+ case (false, false, false, false, _) if hasPrecombineColumn => UPSERT_OPERATION_OPT_VAL
+ // if table is pk table and has enableBulkInsert use bulk insert for non-strict mode.
+ case (true, _, _, true, _) => BULK_INSERT_OPERATION_OPT_VAL
+ // for the rest case, use the insert operation
+ case _ => INSERT_OPERATION_OPT_VAL
+ }
+
+ val payloadClassName = if (operation == UPSERT_OPERATION_OPT_VAL &&
+ tableType == COW_TABLE_TYPE_OPT_VAL && insertMode == InsertMode.STRICT) {
+ // Only validate duplicate key for COW, for MOR it will do the merge with the DefaultHoodieRecordPayload
+ // on reading.
+ classOf[ValidateDuplicateKeyPayload].getCanonicalName
+ } else {
+ classOf[OverwriteWithLatestAvroPayload].getCanonicalName
+ }
+ logInfo(s"insert statement use write operation type: $operation, payloadClass: $payloadClassName")
+
+ val enableHive = isEnableHive(sparkSession)
+ withSparkConf(sparkSession, options) {
+ Map(
+ "path" -> path,
+ TABLE_TYPE.key -> tableType,
+ TBL_NAME.key -> hoodieCatalogTable.tableName,
+ PRECOMBINE_FIELD.key -> preCombineColumn,
+ OPERATION.key -> operation,
+ HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable,
+ URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning,
+ KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
+ SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> keyGeneratorClassName,
+ RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
+ PARTITIONPATH_FIELD.key -> partitionFields,
+ PAYLOAD_CLASS_NAME.key -> payloadClassName,
+ ENABLE_ROW_WRITER.key -> enableBulkInsert.toString,
+ HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn),
+ META_SYNC_ENABLED.key -> enableHive.toString,
+ HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
+ HIVE_USE_JDBC.key -> "false",
+ HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
+ HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table,
+ HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
+ HIVE_PARTITION_FIELDS.key -> partitionFields,
+ HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
+ HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200",
+ HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
+ SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL
+ )
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
new file mode 100644
index 0000000000000..7c98bf549785c
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
@@ -0,0 +1,129 @@
+/*
+ * 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.catalog
+
+import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
+import org.apache.spark.sql.connector.catalog.TableCapability._
+import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, V2TableWithV1Fallback}
+import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform}
+import org.apache.spark.sql.connector.write._
+import org.apache.spark.sql.sources.{Filter, InsertableRelation}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+
+import java.util
+import scala.collection.JavaConverters.{mapAsJavaMapConverter, setAsJavaSetConverter}
+
+case class HoodieInternalV2Table(spark: SparkSession,
+ path: String,
+ catalogTable: Option[CatalogTable] = None,
+ tableIdentifier: Option[String] = None,
+ options: CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty())
+ extends Table with SupportsWrite with V2TableWithV1Fallback {
+
+ lazy val hoodieCatalogTable: HoodieCatalogTable = if (catalogTable.isDefined) {
+ HoodieCatalogTable(spark, catalogTable.get)
+ } else {
+ val metaClient: HoodieTableMetaClient = HoodieTableMetaClient.builder()
+ .setBasePath(path)
+ .setConf(SparkSession.active.sessionState.newHadoopConf)
+ .build()
+
+ val tableConfig: HoodieTableConfig = metaClient.getTableConfig
+ val tableName: String = tableConfig.getTableName
+
+ HoodieCatalogTable(spark, TableIdentifier(tableName))
+ }
+
+ private lazy val tableSchema: StructType = hoodieCatalogTable.tableSchema
+
+ override def name(): String = hoodieCatalogTable.table.identifier.unquotedString
+
+ override def schema(): StructType = tableSchema
+
+ override def capabilities(): util.Set[TableCapability] = Set(
+ BATCH_READ, V1_BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE, ACCEPT_ANY_SCHEMA
+ ).asJava
+
+ override def properties(): util.Map[String, String] = {
+ val map = new util.HashMap[String, String]()
+ map.put("provider", "hudi")
+ map.putAll(hoodieCatalogTable.catalogProperties.asJava)
+ map
+ }
+
+ override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
+ new WriteIntoHoodieBuilder(info.options, hoodieCatalogTable, spark)
+ }
+
+ override def v1Table: CatalogTable = hoodieCatalogTable.table
+
+ override def partitioning(): Array[Transform] = {
+ hoodieCatalogTable.partitionFields.map { col =>
+ new IdentityTransform(new FieldReference(Seq(col)))
+ }.toArray
+ }
+
+}
+
+private class WriteIntoHoodieBuilder(writeOptions: CaseInsensitiveStringMap,
+ hoodieCatalogTable: HoodieCatalogTable,
+ spark: SparkSession)
+ extends SupportsTruncate with SupportsOverwrite with HoodieConfigHelper {
+
+ private var forceOverwrite = false
+
+ override def truncate(): WriteIntoHoodieBuilder = {
+ forceOverwrite = true
+ this
+ }
+
+ override def overwrite(filters: Array[Filter]): WriteBuilder = {
+ forceOverwrite = true
+ this
+ }
+
+ override def build(): V1Write = new V1Write {
+ override def toInsertableRelation: InsertableRelation = {
+ new InsertableRelation {
+ override def insert(data: DataFrame, overwrite: Boolean): Unit = {
+ val mode = if (forceOverwrite && hoodieCatalogTable.partitionFields.isEmpty) {
+ // insert overwrite non-partition table
+ SaveMode.Overwrite
+ } else {
+ // for insert into or insert overwrite partition we use append mode.
+ SaveMode.Append
+ }
+ alignOutputColumns(data).write.format("org.apache.hudi")
+ .mode(mode)
+ .options(buildHoodieConfig(hoodieCatalogTable) ++
+ buildHoodieInsertConfig(hoodieCatalogTable, spark, forceOverwrite, Map.empty, Map.empty))
+ .save()
+ }
+ }
+ }
+ }
+
+ private def alignOutputColumns(data: DataFrame): DataFrame = {
+ val schema = hoodieCatalogTable.tableSchema
+ spark.createDataFrame(data.toJavaRDD, schema)
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
new file mode 100644
index 0000000000000..4034862167aa5
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
@@ -0,0 +1,99 @@
+/*
+ * 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.catalog
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.DataSourceWriteOptions.RECORDKEY_FIELD
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, SupportsWrite, TableCapability}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write, WriteBuilder}
+import org.apache.spark.sql.sources.InsertableRelation
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters.{mapAsScalaMapConverter, setAsJavaSetConverter}
+
+case class HoodieStagedTable(ident: Identifier,
+ catalog: HoodieCatalog,
+ override val schema: StructType,
+ partitions: Array[Transform],
+ override val properties: util.Map[String, String],
+ mode: TableCreationMode) extends StagedTable with SupportsWrite {
+
+ private var sourceQuery: Option[DataFrame] = None
+ private var writeOptions: Map[String, String] = Map.empty
+
+ override def commitStagedChanges(): Unit = {
+ val props = new util.HashMap[String, String]()
+ val optionsThroughProperties = properties.asScala.collect {
+ case (k, _) if k.startsWith("option.") => k.stripPrefix("option.")
+ }.toSet
+ val sqlWriteOptions = new util.HashMap[String, String]()
+ properties.asScala.foreach { case (k, v) =>
+ if (!k.startsWith("option.") && !optionsThroughProperties.contains(k)) {
+ props.put(k, v)
+ } else if (optionsThroughProperties.contains(k)) {
+ sqlWriteOptions.put(k, v)
+ }
+ }
+ if (writeOptions.isEmpty && !sqlWriteOptions.isEmpty) {
+ writeOptions = sqlWriteOptions.asScala.toMap
+ }
+ props.putAll(properties)
+ props.put("hoodie.table.name", ident.name())
+ props.put(RECORDKEY_FIELD.key, properties.get("primaryKey"))
+ catalog.createHoodieTable(ident, schema, partitions, props, writeOptions, sourceQuery, mode)
+ }
+
+ override def name(): String = ident.name()
+
+ override def abortStagedChanges(): Unit = {
+ clearTablePath(properties.get("location"), catalog.spark.sparkContext.hadoopConfiguration)
+ }
+
+ private def clearTablePath(tablePath: String, conf: Configuration): Unit = {
+ val path = new Path(tablePath)
+ val fs = path.getFileSystem(conf)
+ fs.delete(path, true)
+ }
+
+ override def capabilities(): util.Set[TableCapability] = Set(TableCapability.V1_BATCH_WRITE).asJava
+
+ override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
+ writeOptions = info.options.asCaseSensitiveMap().asScala.toMap
+ new HoodieV1WriteBuilder
+ }
+
+ /*
+ * WriteBuilder for creating a Hoodie table.
+ */
+ private class HoodieV1WriteBuilder extends WriteBuilder {
+ override def build(): V1Write = new V1Write {
+ override def toInsertableRelation(): InsertableRelation = {
+ new InsertableRelation {
+ override def insert(data: DataFrame, overwrite: Boolean): Unit = {
+ sourceQuery = Option(data)
+ }
+ }
+ }
+ }
+ }
+
+}
diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
new file mode 100644
index 0000000000000..8b54775be149e
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
@@ -0,0 +1,23 @@
+/*
+ * 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.catalog;
+
+public enum TableCreationMode {
+ CREATE, CREATE_OR_REPLACE, STAGE_CREATE, STAGE_REPLACE
+}
diff --git a/hudi-spark-datasource/pom.xml b/hudi-spark-datasource/pom.xml
index ae7cb8e0b8afa..7aaf39865e054 100644
--- a/hudi-spark-datasource/pom.xml
+++ b/hudi-spark-datasource/pom.xml
@@ -35,5 +35,9 @@
hudi-spark
hudi-spark2
hudi-spark3
+ hudi-spark3.1.x
+ hudi-spark3-common
+ hudi-spark2-common
+
diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml
index 7ab8c21700229..b6d0c8257d746 100644
--- a/packaging/hudi-spark-bundle/pom.xml
+++ b/packaging/hudi-spark-bundle/pom.xml
@@ -60,6 +60,9 @@
META-INF/LICENSE
target/classes/META-INF/LICENSE
+
+ META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+
@@ -69,6 +72,7 @@
org.apache.hudi:hudi-spark-common_${scala.binary.version}
org.apache.hudi:hudi-spark_${scala.binary.version}
org.apache.hudi:${hudi.spark.module}_${scala.binary.version}
+ org.apache.hudi:${hudi.spark.common.module}
org.apache.hudi:hudi-hive-sync
org.apache.hudi:hudi-sync-common
org.apache.hudi:hudi-hadoop-mr
@@ -253,6 +257,11 @@
${hudi.spark.module}_${scala.binary.version}
${project.version}
+
+ org.apache.hudi
+ ${hudi.spark.common.module}
+ ${project.version}
+
org.apache.hudi
hudi-timeline-service
diff --git a/pom.xml b/pom.xml
index a570c1f7149b1..947b0553d4b9c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -123,6 +123,7 @@
3
hudi-spark2
+ hudi-spark2-common
1.8.2
2.11.12
2.12.10
@@ -1577,6 +1578,7 @@
${scala12.version}
2.12
hudi-spark3
+ hudi-spark3-common
3.1.0
2.4.1
${parquet.spark3.version}
@@ -1602,7 +1604,8 @@
${spark3bundle.version}
${scala12.version}
2.12
- hudi-spark3
+ hudi-spark3.1.x
+ hudi-spark3-common
3.1.0
2.4.1
${fasterxml.spark3.version}
@@ -1621,10 +1624,12 @@
spark3.0.x
-
+
3.0.3
${spark3.version}
+ hudi-spark3.1.x
+ hudi-spark3-common
3.0.1