1
0

[HUDI-1929] Support configure KeyGenerator by type (#2993)

This commit is contained in:
wangxianghu
2021-06-08 21:26:10 +08:00
committed by GitHub
parent f760ec543e
commit 7261f08507
20 changed files with 819 additions and 106 deletions

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Dataset;
@@ -62,7 +63,7 @@ public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataPr
Dataset inputDataset = sparkSession.read().parquet(filePaths);
try {
KeyGenerator keyGenerator = DataSourceUtils.createKeyGenerator(props);
KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
String structName = tableName + "_record";
String namespace = "hoodie." + tableName;
RDD<GenericRecord> genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace);

View File

@@ -20,7 +20,6 @@ package org.apache.hudi
import java.util
import java.util.Properties
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
@@ -54,7 +53,8 @@ import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
import scala.collection.JavaConversions._
import scala.collection.mutable.ListBuffer
import org.apache.hudi.common.table.HoodieTableConfig.{DEFAULT_ARCHIVELOG_FOLDER, HOODIE_ARCHIVELOG_FOLDER_PROP_NAME}
import org.apache.hudi.common.table.HoodieTableConfig.{DEFAULT_ARCHIVELOG_FOLDER}
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
object HoodieSparkSqlWriter {
@@ -106,7 +106,7 @@ object HoodieSparkSqlWriter {
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
var tableConfig = getHoodieTableConfig(sparkContext, path.get, hoodieTableConfigOpt)
val keyGenerator = DataSourceUtils.createKeyGenerator(toProperties(parameters))
val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(toProperties(parameters))
if (mode == SaveMode.Ignore && tableExists) {
log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.")

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABL
import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator}
/**
@@ -91,7 +92,7 @@ object HoodieWriterUtils {
def getPartitionColumns(parameters: Map[String, String]): String = {
val props = new TypedProperties()
props.putAll(parameters.asJava)
val keyGen = DataSourceUtils.createKeyGenerator(props)
val keyGen = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props)
getPartitionColumns(keyGen)
}