[HUDI-3172] Refactor hudi existing modules to make more code reuse in V2 Implementation (#4514)
* Introduce hudi-spark3-common and hudi-spark2-common modules to place classes that would be reused in different spark versions, also introduce hudi-spark3.1.x to support spark 3.1.x. * Introduce hudi format under hudi-spark2, hudi-spark3, hudi-spark3.1.x modules and change the hudi format in original hudi-spark module to hudi_v1 format. * Manually tested on Spark 3.1.2 and Spark 3.2.0 SQL. * Added a README.md file under hudi-spark-datasource module.
This commit is contained in:
@@ -44,12 +44,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.
|
||||
|
||||
38
hudi-spark-datasource/README.md
Normal file
38
hudi-spark-datasource/README.md
Normal file
@@ -0,0 +1,38 @@
|
||||
<!--
|
||||
* 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
|
||||
-->
|
||||
|
||||
# Description of the relationship between each module
|
||||
|
||||
This repo contains the code that integrate Hudi with Spark. The repo is split into the following modules
|
||||
|
||||
`hudi-spark`
|
||||
`hudi-spark2`
|
||||
`hudi-spark3`
|
||||
`hudi-spark3.1.x`
|
||||
`hudi-spark2-common`
|
||||
`hudi-spark3-common`
|
||||
`hudi-spark-common`
|
||||
|
||||
* hudi-spark is the module that contains the code that both spark2 & spark3 version would share, also contains the antlr4
|
||||
file that supports spark sql on spark 2.x version.
|
||||
* hudi-spark2 is the module that contains the code that compatible with spark 2.x versions.
|
||||
* hudi-spark3 is the module that contains the code that compatible with spark 3.2.0(and above) versions。
|
||||
* hudi-spark3.1.x is the module that contains the code that compatible with spark3.1.x and spark3.0.x version.
|
||||
* hudi-spark2-common is the module that contains the code that would be reused between spark2.x versions, right now the module
|
||||
has no class since hudi only supports spark 2.4.4 version, and it acts as the placeholder when packaging hudi-spark-bundle module.
|
||||
* hudi-spark3-common is the module that contains the code that would be reused between spark3.x versions.
|
||||
* hudi-spark-common is the module that contains the code that would be reused between spark2.x and spark3.x versions.
|
||||
@@ -170,11 +170,53 @@
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Spark -->
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_${scala.binary.version}</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_${scala.binary.version}</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-hive_${scala.binary.version}</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_${scala.binary.version}</artifactId>
|
||||
<classifier>tests</classifier>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_${scala.binary.version}</artifactId>
|
||||
<classifier>tests</classifier>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-catalyst_${scala.binary.version}</artifactId>
|
||||
<classifier>tests</classifier>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Spark (Packages) -->
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-avro_${scala.binary.version}</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
|
||||
@@ -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,
|
||||
@@ -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,7 +87,7 @@ 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.
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
@@ -489,9 +489,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))
|
||||
}
|
||||
|
||||
@@ -621,7 +621,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 =>
|
||||
@@ -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(",")}")
|
||||
@@ -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))
|
||||
@@ -234,17 +234,17 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
|
||||
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
|
||||
}
|
||||
@@ -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
|
||||
@@ -0,0 +1,304 @@
|
||||
/*
|
||||
* 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.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieMetadataConfig}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstantTimeGenerator}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
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, Expression}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.execution.datasources.LogicalRelation
|
||||
import org.apache.spark.sql.internal.StaticSQLConf
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{Column, DataFrame, SparkSession}
|
||||
|
||||
import java.net.URI
|
||||
import java.text.SimpleDateFormat
|
||||
import java.util.{Locale, Properties}
|
||||
import scala.collection.JavaConverters._
|
||||
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 getTableIdentifier(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)
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -26,12 +26,10 @@ 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
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
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}
|
||||
@@ -27,7 +26,6 @@ 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 scala.util.control.NonFatal
|
||||
@@ -25,13 +25,12 @@ 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(
|
||||
@@ -40,7 +39,7 @@ case class AlterHoodieTableDropPartitionCommand(
|
||||
ifExists : Boolean,
|
||||
purge : Boolean,
|
||||
retainData : Boolean)
|
||||
extends HoodieLeafRunnableCommand {
|
||||
extends HoodieLeafRunnableCommand {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
val fullTableName = s"${tableIdentifier.database}.${tableIdentifier.table}"
|
||||
@@ -25,14 +25,12 @@ 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.{HoodieOptionConfig, HoodieSqlCommonUtils}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive
|
||||
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}
|
||||
@@ -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
|
||||
@@ -34,7 +34,7 @@ import org.apache.spark.sql.types.StringType
|
||||
case class ShowHoodieTablePartitionsCommand(
|
||||
tableIdentifier: TableIdentifier,
|
||||
specOpt: Option[TablePartitionSpec])
|
||||
extends HoodieLeafRunnableCommand {
|
||||
extends HoodieLeafRunnableCommand {
|
||||
|
||||
override val output: Seq[Attribute] = {
|
||||
AttributeReference("partition", StringType, nullable = false)() :: Nil
|
||||
@@ -104,7 +104,7 @@ class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props)
|
||||
}
|
||||
}
|
||||
val timestampFormat = PartitionPathEncodeUtils.escapePathName(
|
||||
SqlKeyGenerator.timestampTimeFormat.print(timeMs))
|
||||
SqlKeyGenerator.timestampTimeFormat.print(timeMs))
|
||||
if (isHiveStyle) s"$hiveStylePrefix$timestampFormat" else timestampFormat
|
||||
case _ => partitionValue
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -242,6 +242,18 @@
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.spark.common.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Logging -->
|
||||
<dependency>
|
||||
<groupId>log4j</groupId>
|
||||
|
||||
@@ -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) =>
|
||||
|
||||
@@ -17,235 +17,14 @@
|
||||
|
||||
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, Cast, Expression, Literal}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{MergeIntoTable, SubqueryAlias}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.{DataType, NullType}
|
||||
|
||||
object HoodieSqlUtils extends SparkAdapterSupport {
|
||||
// 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 +35,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)
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -272,57 +51,11 @@ object HoodieSqlUtils extends SparkAdapterSupport {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,7 +21,6 @@ import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
|
||||
import org.apache.hudi.DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
|
||||
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 +28,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.{getTableIdentifier, getTableLocation, isHoodieTable, removeMetaFields, tableExistsInPath}
|
||||
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, HoodieSqlUtils}
|
||||
import org.apache.spark.sql.types.StringType
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
|
||||
@@ -91,7 +91,7 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
|
||||
// Convert to CompactionHoodieTableCommand
|
||||
case CompactionTable(table, operation, options)
|
||||
if table.resolved && isHoodieTable(table, sparkSession) =>
|
||||
val tableId = getTableIdentify(table)
|
||||
val tableId = getTableIdentifier(table)
|
||||
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableId)
|
||||
CompactionHoodieTableCommand(catalogTable, operation, options)
|
||||
// Convert to CompactionHoodiePathCommand
|
||||
@@ -100,7 +100,7 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
|
||||
// Convert to CompactionShowOnTable
|
||||
case CompactionShowOnTable(table, limit)
|
||||
if isHoodieTable(table, sparkSession) =>
|
||||
val tableId = getTableIdentify(table)
|
||||
val tableId = getTableIdentifier(table)
|
||||
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableId)
|
||||
CompactionShowHoodieTableCommand(catalogTable, limit)
|
||||
// Convert to CompactionShowHoodiePathCommand
|
||||
@@ -161,8 +161,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 +182,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
|
||||
}
|
||||
@@ -333,7 +333,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))
|
||||
@@ -343,9 +343,9 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
|
||||
} else {
|
||||
c
|
||||
}
|
||||
} else {
|
||||
c
|
||||
}
|
||||
} else {
|
||||
c
|
||||
}
|
||||
case p => p
|
||||
}
|
||||
|
||||
@@ -438,15 +438,15 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic
|
||||
// Rewrite the AlterTableAddColumnsCommand to AlterHoodieTableAddColumnsCommand
|
||||
case AlterTableAddColumnsCommand(tableId, colsToAdd)
|
||||
if isHoodieTable(tableId, sparkSession) =>
|
||||
AlterHoodieTableAddColumnsCommand(tableId, colsToAdd)
|
||||
AlterHoodieTableAddColumnsCommand(tableId, colsToAdd)
|
||||
// Rewrite the AlterTableRenameCommand to AlterHoodieTableRenameCommand
|
||||
case AlterTableRenameCommand(oldName, newName, isView)
|
||||
if !isView && isHoodieTable(oldName, sparkSession) =>
|
||||
new AlterHoodieTableRenameCommand(oldName, newName, isView)
|
||||
new AlterHoodieTableRenameCommand(oldName, newName, isView)
|
||||
// Rewrite the AlterTableChangeColumnCommand to AlterHoodieTableChangeColumnCommand
|
||||
case AlterTableChangeColumnCommand(tableName, columnName, newColumn)
|
||||
if isHoodieTable(tableName, sparkSession) =>
|
||||
AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn)
|
||||
AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn)
|
||||
// SPARK-34238: the definition of ShowPartitionsCommand has been changed in Spark3.2.
|
||||
// Match the class type instead of call the `unapply` method.
|
||||
case s: ShowPartitionsCommand
|
||||
|
||||
@@ -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()
|
||||
)
|
||||
|
||||
@@ -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])
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
@@ -34,7 +33,7 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Hoodie
|
||||
|
||||
private val table = deleteTable.table
|
||||
|
||||
private val tableId = getTableIdentify(table)
|
||||
private val tableId = getTableIdentifier(table)
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
logInfo(s"start execute delete command for $tableId")
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -17,32 +17,25 @@
|
||||
|
||||
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.hudi.HoodieSqlUtils.castIfNeeded
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
@@ -173,7 +166,7 @@ object InsertIntoHoodieTableCommand extends Logging {
|
||||
} else { // insert static partitions
|
||||
targetPartitionSchema.fields.map(f => {
|
||||
val staticPartitionValue = staticPartitionValues.getOrElse(f.name,
|
||||
s"Missing static partition value for: ${f.name}")
|
||||
s"Missing static partition value for: ${f.name}")
|
||||
val castAttr = castIfNeeded(Literal.create(staticPartitionValue), f.dataType, conf)
|
||||
Alias(castAttr, f.name)()
|
||||
})
|
||||
@@ -290,21 +283,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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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.{castIfNeeded, 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
|
||||
|
||||
@@ -24,14 +24,14 @@ 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.hudi.HoodieSqlUtils.castIfNeeded
|
||||
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._
|
||||
|
||||
@@ -39,7 +39,7 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends HoodieLeaf
|
||||
with SparkAdapterSupport {
|
||||
|
||||
private val table = updateTable.table
|
||||
private val tableId = getTableIdentify(table)
|
||||
private val tableId = getTableIdentifier(table)
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
logInfo(s"start execute update command for $tableId")
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)
|
||||
|
||||
19
hudi-spark-datasource/hudi-spark2-common/pom.xml
Normal file
19
hudi-spark-datasource/hudi-spark2-common/pom.xml
Normal file
@@ -0,0 +1,19 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hudi-spark-datasource</artifactId>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<version>0.11.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hudi-spark2-common</artifactId>
|
||||
|
||||
<properties>
|
||||
<maven.compiler.source>8</maven.compiler.source>
|
||||
<maven.compiler.target>8</maven.compiler.target>
|
||||
</properties>
|
||||
|
||||
</project>
|
||||
@@ -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
|
||||
@@ -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"
|
||||
}
|
||||
@@ -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))
|
||||
|
||||
247
hudi-spark-datasource/hudi-spark3-common/pom.xml
Normal file
247
hudi-spark-datasource/hudi-spark3-common/pom.xml
Normal file
@@ -0,0 +1,247 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hudi-spark-datasource</artifactId>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<version>0.11.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
|
||||
<properties>
|
||||
<main.basedir>${project.parent.parent.basedir}</main.basedir>
|
||||
<maven.compiler.source>8</maven.compiler.source>
|
||||
<maven.compiler.target>8</maven.compiler.target>
|
||||
</properties>
|
||||
|
||||
<build>
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/resources</directory>
|
||||
</resource>
|
||||
</resources>
|
||||
<pluginManagement>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<version>${scala-maven-plugin.version}</version>
|
||||
<configuration>
|
||||
<args>
|
||||
<arg>-nobootcp</arg>
|
||||
</args>
|
||||
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>copy-dependencies</id>
|
||||
<phase>prepare-package</phase>
|
||||
<goals>
|
||||
<goal>copy-dependencies</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputDirectory>${project.build.directory}/lib</outputDirectory>
|
||||
<overWriteReleases>true</overWriteReleases>
|
||||
<overWriteSnapshots>true</overWriteSnapshots>
|
||||
<overWriteIfNewer>true</overWriteIfNewer>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
<phase>process-resources</phase>
|
||||
<goals>
|
||||
<goal>add-source</goal>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>scala-test-compile</id>
|
||||
<phase>process-test-resources</phase>
|
||||
<goals>
|
||||
<goal>testCompile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<source>1.8</source>
|
||||
<target>1.8</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>compile</phase>
|
||||
<goals>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<source>1.8</source>
|
||||
<target>1.8</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
<phase>test-compile</phase>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<skip>false</skip>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<configuration>
|
||||
<skipTests>${skip.hudi-spark3.unit.tests}</skipTests>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.scalastyle</groupId>
|
||||
<artifactId>scalastyle-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.scala-lang</groupId>
|
||||
<artifactId>scala-library</artifactId>
|
||||
<version>${scala12.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.12</artifactId>
|
||||
<version>${spark3.version}</version>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<version>${fasterxml.spark3.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
<version>${fasterxml.spark3.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
<version>${fasterxml.spark3.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-client-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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/".
|
||||
242
hudi-spark-datasource/hudi-spark3.1.x/pom.xml
Normal file
242
hudi-spark-datasource/hudi-spark3.1.x/pom.xml
Normal file
@@ -0,0 +1,242 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hudi-spark-datasource</artifactId>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<version>0.11.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hudi-spark3.1.x_2.12</artifactId>
|
||||
<version>0.11.0-SNAPSHOT</version>
|
||||
|
||||
<name>hudi-spark3.1.x_2.12</name>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<main.basedir>${project.parent.parent.basedir}</main.basedir>
|
||||
</properties>
|
||||
|
||||
<build>
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/resources</directory>
|
||||
</resource>
|
||||
</resources>
|
||||
<pluginManagement>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<version>${scala-maven-plugin.version}</version>
|
||||
<configuration>
|
||||
<args>
|
||||
<arg>-nobootcp</arg>
|
||||
</args>
|
||||
<checkMultipleScalaVersions>false</checkMultipleScalaVersions>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>copy-dependencies</id>
|
||||
<phase>prepare-package</phase>
|
||||
<goals>
|
||||
<goal>copy-dependencies</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputDirectory>${project.build.directory}/lib</outputDirectory>
|
||||
<overWriteReleases>true</overWriteReleases>
|
||||
<overWriteSnapshots>true</overWriteSnapshots>
|
||||
<overWriteIfNewer>true</overWriteIfNewer>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
<phase>process-resources</phase>
|
||||
<goals>
|
||||
<goal>add-source</goal>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>scala-test-compile</id>
|
||||
<phase>process-test-resources</phase>
|
||||
<goals>
|
||||
<goal>testCompile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>compile</phase>
|
||||
<goals>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
<phase>test-compile</phase>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<skip>false</skip>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<configuration>
|
||||
<skipTests>${skip.hudi-spark3.unit.tests}</skipTests>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.scalastyle</groupId>
|
||||
<artifactId>scalastyle-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.scala-lang</groupId>
|
||||
<artifactId>scala-library</artifactId>
|
||||
<version>${scala12.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.12</artifactId>
|
||||
<version>${spark3.version}</version>
|
||||
<optional>true</optional>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<version>${fasterxml.spark3.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
<version>${fasterxml.spark3.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
<version>${fasterxml.spark3.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-client-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
||||
@@ -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
|
||||
@@ -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"
|
||||
}
|
||||
@@ -187,6 +187,11 @@
|
||||
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-spark3-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Test -->
|
||||
<dependency>
|
||||
|
||||
@@ -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
|
||||
@@ -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 Spark3DefaultSource extends DefaultSource with DataSourceRegister {
|
||||
override def shortName(): String = "hudi"
|
||||
}
|
||||
@@ -35,5 +35,9 @@
|
||||
<module>hudi-spark</module>
|
||||
<module>hudi-spark2</module>
|
||||
<module>hudi-spark3</module>
|
||||
<module>hudi-spark3.1.x</module>
|
||||
<module>hudi-spark3-common</module>
|
||||
<module>hudi-spark2-common</module>
|
||||
</modules>
|
||||
|
||||
</project>
|
||||
|
||||
@@ -141,6 +141,12 @@
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.spark.common.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Kafka -->
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
|
||||
@@ -74,6 +74,7 @@
|
||||
<include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:${hudi.spark.module}_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:${hudi.spark.common.module}</include>
|
||||
<include>org.apache.hudi:hudi-hive-sync</include>
|
||||
<include>org.apache.hudi:hudi-sync-common</include>
|
||||
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
||||
@@ -350,6 +351,12 @@
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.spark.common.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
|
||||
@@ -60,6 +60,9 @@
|
||||
<resource>META-INF/LICENSE</resource>
|
||||
<file>target/classes/META-INF/LICENSE</file>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
|
||||
<resource>META-INF/services/org.apache.spark.sql.sources.DataSourceRegister</resource>
|
||||
</transformer>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -69,6 +72,7 @@
|
||||
<include>org.apache.hudi:hudi-spark-common_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:${hudi.spark.module}_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:${hudi.spark.common.module}</include>
|
||||
<include>org.apache.hudi:hudi-hive-sync</include>
|
||||
<include>org.apache.hudi:hudi-sync-common</include>
|
||||
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
||||
@@ -249,6 +253,11 @@
|
||||
<artifactId>${hudi.spark.module}_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.spark.common.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-timeline-service</artifactId>
|
||||
|
||||
@@ -93,6 +93,7 @@
|
||||
<include>org.apache.hudi:hudi-spark-common_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:${hudi.spark.module}_${scala.binary.version}</include>
|
||||
<include>org.apache.hudi:${hudi.spark.common.module}</include>
|
||||
<include>org.apache.hudi:hudi-hive-sync</include>
|
||||
<include>org.apache.hudi:hudi-sync-common</include>
|
||||
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
||||
@@ -275,6 +276,11 @@
|
||||
<artifactId>${hudi.spark.module}_${scala.binary.version}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>${hudi.spark.common.module}</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-utilities_${scala.binary.version}</artifactId>
|
||||
|
||||
5
pom.xml
5
pom.xml
@@ -121,6 +121,7 @@
|
||||
<spark2.version>2.4.4</spark2.version>
|
||||
<spark3.version>3.2.0</spark3.version>
|
||||
<hudi.spark.module>hudi-spark2</hudi.spark.module>
|
||||
<hudi.spark.common.module>hudi-spark2-common</hudi.spark.common.module>
|
||||
<avro.version>1.8.2</avro.version>
|
||||
<scala11.version>2.11.12</scala11.version>
|
||||
<scala12.version>2.12.10</scala12.version>
|
||||
@@ -1582,6 +1583,7 @@
|
||||
<scala.version>${scala12.version}</scala.version>
|
||||
<scala.binary.version>2.12</scala.binary.version>
|
||||
<hudi.spark.module>hudi-spark3</hudi.spark.module>
|
||||
<hudi.spark.common.module>hudi-spark3-common</hudi.spark.common.module>
|
||||
<scalatest.version>3.1.0</scalatest.version>
|
||||
<kafka.version>2.4.1</kafka.version>
|
||||
<parquet.version>1.12.1</parquet.version>
|
||||
@@ -1607,7 +1609,8 @@
|
||||
<sparkbundle.version>${spark3.version}</sparkbundle.version>
|
||||
<scala.version>${scala12.version}</scala.version>
|
||||
<scala.binary.version>2.12</scala.binary.version>
|
||||
<hudi.spark.module>hudi-spark3</hudi.spark.module>
|
||||
<hudi.spark.module>hudi-spark3.1.x</hudi.spark.module>
|
||||
<hudi.spark.common.module>hudi-spark3-common</hudi.spark.common.module>
|
||||
<scalatest.version>3.1.0</scalatest.version>
|
||||
<kafka.version>2.4.1</kafka.version>
|
||||
<fasterxml.version>${fasterxml.spark3.version}</fasterxml.version>
|
||||
|
||||
Reference in New Issue
Block a user