[HUDI-1104] Adding support for UserDefinedPartitioners and SortModes to BulkInsert with Rows (#3149)
This commit is contained in:
committed by
GitHub
parent
55ecbc662e
commit
ea9e5d0e8b
@@ -18,17 +18,12 @@
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import static org.apache.spark.sql.functions.callUDF;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -40,8 +35,16 @@ import org.apache.spark.sql.api.java.UDF1;
|
||||
import org.apache.spark.sql.functions;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
import static org.apache.spark.sql.functions.callUDF;
|
||||
|
||||
/**
|
||||
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
|
||||
*/
|
||||
@@ -60,12 +63,13 @@ public class HoodieDatasetBulkInsertHelper {
|
||||
* 4. Sorts input dataset by hoodie partition path and record key
|
||||
*
|
||||
* @param sqlContext SQL Context
|
||||
* @param config Hoodie Write Config
|
||||
* @param rows Spark Input dataset
|
||||
* @param config Hoodie Write Config
|
||||
* @param rows Spark Input dataset
|
||||
* @return hoodie dataset which is ready for bulk insert.
|
||||
*/
|
||||
public static Dataset<Row> prepareHoodieDatasetForBulkInsert(SQLContext sqlContext,
|
||||
HoodieWriteConfig config, Dataset<Row> rows, String structName, String recordNamespace) {
|
||||
HoodieWriteConfig config, Dataset<Row> rows, String structName, String recordNamespace,
|
||||
BulkInsertPartitioner<Dataset<Row>> bulkInsertPartitionerRows) {
|
||||
List<Column> originalFields =
|
||||
Arrays.stream(rows.schema().fields()).map(f -> new Column(f.name())).collect(Collectors.toList());
|
||||
|
||||
@@ -101,8 +105,6 @@ public class HoodieDatasetBulkInsertHelper {
|
||||
Dataset<Row> colOrderedDataset = rowDatasetWithHoodieColumns.select(
|
||||
JavaConverters.collectionAsScalaIterableConverter(orderedFields).asScala().toSeq());
|
||||
|
||||
return colOrderedDataset
|
||||
.sort(functions.col(HoodieRecord.PARTITION_PATH_METADATA_FIELD), functions.col(HoodieRecord.RECORD_KEY_METADATA_FIELD))
|
||||
.coalesce(config.getBulkInsertShuffleParallelism());
|
||||
return bulkInsertPartitionerRows.repartitionRecords(colOrderedDataset, config.getBulkInsertShuffleParallelism());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,7 +19,6 @@ package org.apache.hudi
|
||||
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
@@ -34,13 +33,15 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory
|
||||
import org.apache.hudi.hive.util.ConfigUtils
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||
import org.apache.hudi.table.BulkInsertPartitioner
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.SparkContext
|
||||
@@ -50,7 +51,7 @@ import org.apache.spark.sql.hudi.HoodieSqlUtils
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
@@ -335,7 +336,17 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
|
||||
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
|
||||
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
|
||||
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
|
||||
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (userDefinedBulkInsertPartitionerOpt.isPresent) {
|
||||
userDefinedBulkInsertPartitionerOpt.get
|
||||
}
|
||||
else {
|
||||
BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig.getBulkInsertSortMode)
|
||||
}
|
||||
val arePartitionRecordsSorted = bulkInsertPartitionerRows.arePartitionRecordsSorted();
|
||||
parameters.updated(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, arePartitionRecordsSorted.toString)
|
||||
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace,
|
||||
bulkInsertPartitionerRows)
|
||||
if (SPARK_VERSION.startsWith("2.")) {
|
||||
hoodieDF.write.format("org.apache.hudi.internal")
|
||||
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
|
||||
|
||||
Reference in New Issue
Block a user