[HUDI-1013] Adding Bulk Insert V2 implementation (#1834)
- Adding ability to use native spark row writing for bulk_insert - Controlled by `ENABLE_ROW_WRITER_OPT_KEY` datasource write option - Introduced KeyGeneratorInterface in hudi-client, moved KeyGenerator back to hudi-spark - Simplified the new API additions to just two new methods : getRecordKey(row), getPartitionPath(row) - Fixed all built-in key generators with new APIs - Made the field position map lazily created upon the first call to row based apis - Implemented native row based key generators for CustomKeyGenerator - Fixed all the tests, with these new APIs Co-authored-by: Balaji Varadarajan <varadarb@uber.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
8d04268264
commit
379cf0786f
@@ -18,8 +18,6 @@
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
@@ -48,6 +46,8 @@ import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -106,7 +106,7 @@ public class DataSourceUtils {
|
||||
|
||||
public static String getTablePath(FileSystem fs, Path[] userProvidedPaths) throws IOException {
|
||||
LOG.info("Getting table path..");
|
||||
for (Path path: userProvidedPaths) {
|
||||
for (Path path : userProvidedPaths) {
|
||||
try {
|
||||
Option<Path> tablePath = TablePathUtils.getTablePath(fs, path);
|
||||
if (tablePath.isPresent()) {
|
||||
@@ -123,8 +123,7 @@ public class DataSourceUtils {
|
||||
/**
|
||||
* This method converts values for fields with certain Avro/Parquet data types that require special handling.
|
||||
*
|
||||
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is
|
||||
* represented/stored in parquet.
|
||||
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is represented/stored in parquet.
|
||||
*
|
||||
* @param fieldSchema avro field schema
|
||||
* @param fieldValue avro field value
|
||||
@@ -157,9 +156,8 @@ public class DataSourceUtils {
|
||||
/**
|
||||
* Create a key generator class via reflection, passing in any configs needed.
|
||||
* <p>
|
||||
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the
|
||||
* corresponding key generator class; otherwise, use the default key generator class specified in {@code
|
||||
* DataSourceWriteOptions}.
|
||||
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the corresponding key generator class; otherwise, use the default key generator class
|
||||
* specified in {@code DataSourceWriteOptions}.
|
||||
*/
|
||||
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
|
||||
String keyGeneratorClass = props.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
@@ -173,10 +171,6 @@ public class DataSourceUtils {
|
||||
|
||||
/**
|
||||
* Create a date time parser class for TimestampBasedKeyGenerator, passing in any configs needed.
|
||||
* @param props
|
||||
* @param parserClass
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HoodieDateTimeParser createDateTimeParser(TypedProperties props, String parserClass) throws IOException {
|
||||
try {
|
||||
@@ -190,6 +184,7 @@ public class DataSourceUtils {
|
||||
* Create a UserDefinedBulkInsertPartitioner class via reflection,
|
||||
* <br>
|
||||
* if the class name of UserDefinedBulkInsertPartitioner is configured through the HoodieWriteConfig.
|
||||
*
|
||||
* @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass()
|
||||
*/
|
||||
private static Option<BulkInsertPartitioner> createUserDefinedBulkInsertPartitioner(HoodieWriteConfig config)
|
||||
@@ -225,35 +220,35 @@ public class DataSourceUtils {
|
||||
});
|
||||
}
|
||||
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
|
||||
public static HoodieWriteConfig createHoodieConfig(String schemaStr, String basePath,
|
||||
String tblName, Map<String, String> parameters) {
|
||||
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY()));
|
||||
// inline compaction is on by default for MOR
|
||||
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY()));
|
||||
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY())
|
||||
.equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
|
||||
return createHoodieClient(jssc, schemaStr, basePath, tblName, parameters, inlineCompact);
|
||||
}
|
||||
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
|
||||
String tblName, Map<String, String> parameters, boolean inlineCompact) {
|
||||
|
||||
// insert/bulk-insert combining to be true, if filtering for duplicates
|
||||
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY()));
|
||||
HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath).withAutoCommit(false).combineInput(combineInserts, true);
|
||||
if (schemaStr != null) {
|
||||
builder = builder.withSchema(schemaStr);
|
||||
}
|
||||
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath).withAutoCommit(false)
|
||||
.combineInput(combineInserts, true).withSchema(schemaStr).forTable(tblName)
|
||||
return builder.forTable(tblName)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY()))
|
||||
.withInlineCompaction(inlineCompact).build())
|
||||
// override above with Hoodie configs specified as options.
|
||||
.withProps(parameters).build();
|
||||
}
|
||||
|
||||
return new HoodieWriteClient<>(jssc, writeConfig, true);
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
|
||||
String tblName, Map<String, String> parameters) {
|
||||
return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
|
||||
}
|
||||
|
||||
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
|
||||
String instantTime, String operation) throws HoodieException {
|
||||
String instantTime, String operation) throws HoodieException {
|
||||
if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) {
|
||||
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner =
|
||||
createUserDefinedBulkInsertPartitioner(client.getConfig());
|
||||
@@ -267,12 +262,12 @@ public class DataSourceUtils {
|
||||
}
|
||||
|
||||
public static JavaRDD<WriteStatus> doDeleteOperation(HoodieWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
|
||||
String instantTime) {
|
||||
String instantTime) {
|
||||
return client.delete(hoodieKeys, instantTime);
|
||||
}
|
||||
|
||||
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey,
|
||||
String payloadClass) throws IOException {
|
||||
String payloadClass) throws IOException {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
|
||||
return new HoodieRecord<>(hKey, payload);
|
||||
}
|
||||
@@ -280,13 +275,13 @@ public class DataSourceUtils {
|
||||
/**
|
||||
* Drop records already present in the dataset.
|
||||
*
|
||||
* @param jssc JavaSparkContext
|
||||
* @param jssc JavaSparkContext
|
||||
* @param incomingHoodieRecords HoodieRecords to deduplicate
|
||||
* @param writeConfig HoodieWriteConfig
|
||||
* @param writeConfig HoodieWriteConfig
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
HoodieWriteConfig writeConfig) {
|
||||
try {
|
||||
HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig);
|
||||
return client.tagLocation(incomingHoodieRecords)
|
||||
@@ -300,7 +295,7 @@ public class DataSourceUtils {
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
Map<String, String> parameters) {
|
||||
Map<String, String> parameters) {
|
||||
HoodieWriteConfig writeConfig =
|
||||
HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build();
|
||||
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
|
||||
|
||||
@@ -0,0 +1,108 @@
|
||||
/*
|
||||
* 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 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.KeyGenerator;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.sql.Column;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
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 scala.collection.JavaConverters;
|
||||
|
||||
/**
|
||||
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
|
||||
*/
|
||||
public class HoodieDatasetBulkInsertHelper {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieDatasetBulkInsertHelper.class);
|
||||
|
||||
private static final String RECORD_KEY_UDF_FN = "hudi_recordkey_gen_function";
|
||||
private static final String PARTITION_PATH_UDF_FN = "hudi_partition_gen_function";
|
||||
|
||||
/**
|
||||
* Prepares input hoodie spark dataset for bulk insert. It does the following steps.
|
||||
* 1. Uses KeyGenerator to generate hoodie record keys and partition path.
|
||||
* 2. Add hoodie columns to input spark dataset.
|
||||
* 3. Reorders input dataset columns so that hoodie columns appear in the beginning.
|
||||
* 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
|
||||
* @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) {
|
||||
List<Column> originalFields =
|
||||
Arrays.stream(rows.schema().fields()).map(f -> new Column(f.name())).collect(Collectors.toList());
|
||||
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(config.getProps());
|
||||
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY());
|
||||
KeyGenerator keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
|
||||
StructType structTypeForUDF = rows.schema();
|
||||
|
||||
sqlContext.udf().register(RECORD_KEY_UDF_FN, (UDF1<Row, String>) keyGenerator::getRecordKey, DataTypes.StringType);
|
||||
sqlContext.udf().register(PARTITION_PATH_UDF_FN, (UDF1<Row, String>) keyGenerator::getPartitionPath, DataTypes.StringType);
|
||||
|
||||
final Dataset<Row> rowDatasetWithRecordKeys = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD,
|
||||
callUDF(RECORD_KEY_UDF_FN, org.apache.spark.sql.functions.struct(
|
||||
JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq())));
|
||||
|
||||
final Dataset<Row> rowDatasetWithRecordKeysAndPartitionPath =
|
||||
rowDatasetWithRecordKeys.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
|
||||
callUDF(PARTITION_PATH_UDF_FN,
|
||||
org.apache.spark.sql.functions.struct(
|
||||
JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq())));
|
||||
|
||||
// Add other empty hoodie fields which will be populated before writing to parquet.
|
||||
Dataset<Row> rowDatasetWithHoodieColumns =
|
||||
rowDatasetWithRecordKeysAndPartitionPath.withColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType));
|
||||
List<Column> orderedFields = Stream.concat(HoodieRecord.HOODIE_META_COLUMNS.stream().map(Column::new),
|
||||
originalFields.stream()).collect(Collectors.toList());
|
||||
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());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* 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.internal;
|
||||
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.sources.DataSourceRegister;
|
||||
import org.apache.spark.sql.sources.v2.DataSourceOptions;
|
||||
import org.apache.spark.sql.sources.v2.DataSourceV2;
|
||||
import org.apache.spark.sql.sources.v2.ReadSupport;
|
||||
import org.apache.spark.sql.sources.v2.WriteSupport;
|
||||
import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* DataSource V2 implementation for managing internal write logic. Only called internally.
|
||||
*/
|
||||
public class DefaultSource implements DataSourceV2, ReadSupport, WriteSupport,
|
||||
DataSourceRegister {
|
||||
|
||||
private static final Logger LOG = LogManager
|
||||
.getLogger(DefaultSource.class);
|
||||
|
||||
private SparkSession sparkSession = null;
|
||||
private Configuration configuration = null;
|
||||
|
||||
@Override
|
||||
public String shortName() {
|
||||
return "hudi_internal";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSourceReader createReader(StructType schema, DataSourceOptions options) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSourceReader createReader(DataSourceOptions options) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<DataSourceWriter> createWriter(String writeUUID, StructType schema, SaveMode mode,
|
||||
DataSourceOptions options) {
|
||||
String instantTime = options.get(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY).get();
|
||||
String path = options.get("path").get();
|
||||
String tblName = options.get(HoodieWriteConfig.TABLE_NAME).get();
|
||||
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, options.asMap());
|
||||
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
|
||||
getConfiguration()));
|
||||
}
|
||||
|
||||
private SparkSession getSparkSession() {
|
||||
if (sparkSession == null) {
|
||||
sparkSession = SparkSession.builder().getOrCreate();
|
||||
}
|
||||
return sparkSession;
|
||||
}
|
||||
|
||||
private Configuration getConfiguration() {
|
||||
if (configuration == null) {
|
||||
this.configuration = getSparkSession().sparkContext().hadoopConfiguration();
|
||||
}
|
||||
return configuration;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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.internal;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieRowCreateHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataWriter;
|
||||
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* Hoodie's Implementation of {@link DataWriter<InternalRow>}. This is used in data source implementation for bulk insert.
|
||||
*/
|
||||
public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRow> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieBulkInsertDataInternalWriter.class);
|
||||
|
||||
private final String instantTime;
|
||||
private final int taskPartitionId;
|
||||
private final long taskId;
|
||||
private final long taskEpochId;
|
||||
private final HoodieTable hoodieTable;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
|
||||
|
||||
private HoodieRowCreateHandle handle;
|
||||
private String lastKnownPartitionPath = null;
|
||||
private String fileIdPrefix = null;
|
||||
private int numFilesWritten = 0;
|
||||
|
||||
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
|
||||
StructType structType) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.writeConfig = writeConfig;
|
||||
this.instantTime = instantTime;
|
||||
this.taskPartitionId = taskPartitionId;
|
||||
this.taskId = taskId;
|
||||
this.taskEpochId = taskEpochId;
|
||||
this.structType = structType;
|
||||
this.fileIdPrefix = UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(InternalRow record) throws IOException {
|
||||
try {
|
||||
String partitionPath = record.getUTF8String(
|
||||
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
|
||||
|
||||
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
|
||||
LOG.info("Creating new file for partition path " + partitionPath);
|
||||
createNewHandle(partitionPath);
|
||||
lastKnownPartitionPath = partitionPath;
|
||||
}
|
||||
handle.write(record);
|
||||
} catch (Throwable t) {
|
||||
LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t);
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriterCommitMessage commit() throws IOException {
|
||||
close();
|
||||
return new HoodieWriterCommitMessage(writeStatusList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() throws IOException {
|
||||
}
|
||||
|
||||
private void createNewHandle(String partitionPath) throws IOException {
|
||||
if (null != handle) {
|
||||
close();
|
||||
}
|
||||
handle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
|
||||
instantTime, taskPartitionId, taskId, taskEpochId, structType);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
if (null != handle) {
|
||||
writeStatusList.add(handle.close());
|
||||
}
|
||||
}
|
||||
|
||||
protected String getNextFileId() {
|
||||
return String.format("%s-%d", fileIdPrefix, numFilesWritten++);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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.internal;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataWriter;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
/**
|
||||
* Factory to assist in instantiating {@link HoodieBulkInsertDataInternalWriter}.
|
||||
*/
|
||||
public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFactory<InternalRow> {
|
||||
|
||||
private final String instantTime;
|
||||
private final HoodieTable hoodieTable;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
|
||||
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||
String instantTime, StructType structType) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.writeConfig = writeConfig;
|
||||
this.instantTime = instantTime;
|
||||
this.structType = structType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
|
||||
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId,
|
||||
structType);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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.internal;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
|
||||
import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
|
||||
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
/**
|
||||
* Implementation of {@link DataSourceWriter} for datasource "hudi.internal" to be used in datasource implementation
|
||||
* of bulk insert.
|
||||
*/
|
||||
public class HoodieDataSourceInternalWriter implements DataSourceWriter {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieDataSourceInternalWriter.class);
|
||||
public static final String INSTANT_TIME_OPT_KEY = "hoodie.instant.time";
|
||||
|
||||
private final String instantTime;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final StructType structType;
|
||||
private final HoodieWriteClient writeClient;
|
||||
private final HoodieTable hoodieTable;
|
||||
private final WriteOperationType operationType;
|
||||
|
||||
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
||||
SparkSession sparkSession, Configuration configuration) {
|
||||
this.instantTime = instantTime;
|
||||
this.writeConfig = writeConfig;
|
||||
this.structType = structType;
|
||||
this.operationType = WriteOperationType.BULK_INSERT;
|
||||
this.writeClient = new HoodieWriteClient<>(new JavaSparkContext(sparkSession.sparkContext()), writeConfig, true);
|
||||
writeClient.setOperationType(operationType);
|
||||
writeClient.startCommitWithTime(instantTime);
|
||||
this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath());
|
||||
this.hoodieTable = HoodieTable.create(metaClient, writeConfig, metaClient.getHadoopConf());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataWriterFactory<InternalRow> createWriterFactory() {
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty());
|
||||
if (WriteOperationType.BULK_INSERT == operationType) {
|
||||
return new HoodieBulkInsertDataInternalWriterFactory(hoodieTable, writeConfig, instantTime, structType);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Write Operation Type + " + operationType + " not supported ");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCommitCoordinator() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onDataWriterCommit(WriterCommitMessage message) {
|
||||
LOG.info("Received commit of a data writer =" + message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commit(WriterCommitMessage[] messages) {
|
||||
List<HoodieWriteStat> writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m)
|
||||
.flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList());
|
||||
|
||||
try {
|
||||
writeClient.commitStats(instantTime, writeStatList, Option.empty());
|
||||
} catch (Exception ioe) {
|
||||
throw new HoodieException(ioe.getMessage(), ioe);
|
||||
} finally {
|
||||
writeClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(WriterCommitMessage[] messages) {
|
||||
LOG.error("Commit " + instantTime + " aborted ");
|
||||
writeClient.rollback(instantTime);
|
||||
writeClient.close();
|
||||
}
|
||||
}
|
||||
@@ -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.hudi.internal;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
|
||||
|
||||
/**
|
||||
* Hoodie's {@link WriterCommitMessage} used in datasource implementation.
|
||||
*/
|
||||
public class HoodieWriterCommitMessage implements WriterCommitMessage {
|
||||
|
||||
private List<HoodieInternalWriteStatus> writeStatuses = new ArrayList<>();
|
||||
|
||||
public HoodieWriterCommitMessage(List<HoodieInternalWriteStatus> writeStatuses) {
|
||||
this.writeStatuses = writeStatuses;
|
||||
}
|
||||
|
||||
public List<HoodieInternalWriteStatus> getWriteStatuses() {
|
||||
return writeStatuses;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieWriterCommitMessage{" + "writeStatuses=" + writeStatuses + '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,130 @@
|
||||
/*
|
||||
* 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.keygen;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Base class for all the built-in key generators. Contains methods structured for
|
||||
* code reuse amongst them.
|
||||
*/
|
||||
public abstract class BuiltinKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected List<String> recordKeyFields;
|
||||
protected List<String> partitionPathFields;
|
||||
protected final boolean encodePartitionPath;
|
||||
protected final boolean hiveStylePartitioning;
|
||||
|
||||
protected Map<String, List<Integer>> recordKeyPositions = new HashMap<>();
|
||||
protected Map<String, List<Integer>> partitionPathPositions = new HashMap<>();
|
||||
protected StructType structType;
|
||||
|
||||
protected BuiltinKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.encodePartitionPath = config.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
|
||||
this.hiveStylePartitioning = config.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a record Key out of provided generic record.
|
||||
*/
|
||||
public abstract String getRecordKey(GenericRecord record);
|
||||
|
||||
/**
|
||||
* Generate a partition path out of provided generic record.
|
||||
*/
|
||||
public abstract String getPartitionPath(GenericRecord record);
|
||||
|
||||
/**
|
||||
* Generate a Hoodie Key out of provided generic record.
|
||||
*/
|
||||
public final HoodieKey getKey(GenericRecord record) {
|
||||
if (getRecordKeyFields() == null || getPartitionPathFields() == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
return new HoodieKey(getRecordKey(record), getPartitionPath(record));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final List<String> getRecordKeyFieldNames() {
|
||||
// For nested columns, pick top level column name
|
||||
return getRecordKeyFields().stream().map(k -> {
|
||||
int idx = k.indexOf('.');
|
||||
return idx > 0 ? k.substring(0, idx) : k;
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
void buildFieldPositionMapIfNeeded(StructType structType) {
|
||||
if (this.structType == null) {
|
||||
// parse simple fields
|
||||
getRecordKeyFields().stream()
|
||||
.filter(f -> !(f.contains(".")))
|
||||
.forEach(f -> {
|
||||
if (structType.getFieldIndex(f).isDefined()) {
|
||||
recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
|
||||
} else {
|
||||
throw new HoodieKeyException("recordKey value not found for field: \"" + f + "\"");
|
||||
}
|
||||
});
|
||||
// parse nested fields
|
||||
getRecordKeyFields().stream()
|
||||
.filter(f -> f.contains("."))
|
||||
.forEach(f -> recordKeyPositions.put(f, RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, true)));
|
||||
// parse simple fields
|
||||
if (getPartitionPathFields() != null) {
|
||||
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains(".")))
|
||||
.forEach(f -> {
|
||||
if (structType.getFieldIndex(f).isDefined()) {
|
||||
partitionPathPositions.put(f,
|
||||
Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
|
||||
} else {
|
||||
partitionPathPositions.put(f, Collections.singletonList(-1));
|
||||
}
|
||||
});
|
||||
// parse nested fields
|
||||
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains("."))
|
||||
.forEach(f -> partitionPathPositions.put(f,
|
||||
RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false)));
|
||||
}
|
||||
this.structType = structType;
|
||||
}
|
||||
}
|
||||
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
}
|
||||
}
|
||||
@@ -24,8 +24,8 @@ import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
/**
|
||||
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
@@ -34,39 +34,34 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":";
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
protected final List<String> partitionPathFields;
|
||||
protected final boolean hiveStylePartitioning;
|
||||
protected final boolean encodePartitionPath;
|
||||
|
||||
public ComplexKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields =
|
||||
Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
|
||||
this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
|
||||
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY())
|
||||
.split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields = Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY())
|
||||
.split(",")).map(String::trim).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordKey(record, recordKeyFields);
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordPartitionPath(record, partitionPathFields, hiveStylePartitioning, encodePartitionPath);
|
||||
return KeyGenUtils.getRecordPartitionPath(record, getPartitionPathFields(), hiveStylePartitioning, encodePartitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
public String getRecordKey(Row row) {
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
public String getPartitionPath(Row row) {
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
|
||||
hiveStylePartitioning, partitionPathPositions);
|
||||
}
|
||||
}
|
||||
@@ -20,37 +20,32 @@ package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieDeltaStreamerException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
|
||||
* Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
|
||||
* "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
|
||||
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields. Similarly partition path can be configured to have multiple
|
||||
* fields or only one field. This class expects value for prop "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
|
||||
*
|
||||
* properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
|
||||
*
|
||||
* The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
|
||||
*
|
||||
* Few points to consider:
|
||||
* 1. If you want to customize some partition path field on a timestamp basis, you can use field1:timestampBased
|
||||
* 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
|
||||
* 3. If you want your table to be non partitioned, simply leave it as blank.
|
||||
* Few points to consider: 1. If you want to customize some partition path field on a timestamp basis, you can use field1:timestampBased 2. If you simply want to have the value of your configured
|
||||
* field in the partition path, use field1:simple 3. If you want your table to be non partitioned, simply leave it as blank.
|
||||
*
|
||||
* RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
|
||||
*/
|
||||
public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
protected final List<String> partitionPathFields;
|
||||
protected final TypedProperties properties;
|
||||
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
private static final String SPLIT_REGEX = ":";
|
||||
|
||||
@@ -63,15 +58,22 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
public CustomKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.properties = props;
|
||||
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields =
|
||||
Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields = Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(Row row) {
|
||||
return getPartitionPath(Option.empty(), Option.of(row));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(GenericRecord record) {
|
||||
if (partitionPathFields == null) {
|
||||
return getPartitionPath(Option.of(record), Option.empty());
|
||||
}
|
||||
|
||||
private String getPartitionPath(Option<GenericRecord> record, Option<Row> row) {
|
||||
if (getPartitionPathFields() == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for partition path in cfg");
|
||||
}
|
||||
|
||||
@@ -79,10 +81,10 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
|
||||
//Corresponds to no partition case
|
||||
if (partitionPathFields.size() == 1 && partitionPathFields.get(0).isEmpty()) {
|
||||
if (getPartitionPathFields().size() == 1 && getPartitionPathFields().get(0).isEmpty()) {
|
||||
return "";
|
||||
}
|
||||
for (String field : partitionPathFields) {
|
||||
for (String field : getPartitionPathFields()) {
|
||||
String[] fieldWithType = field.split(SPLIT_REGEX);
|
||||
if (fieldWithType.length != 2) {
|
||||
throw new HoodieKeyException("Unable to find field names for partition path in proper format");
|
||||
@@ -92,11 +94,19 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
|
||||
switch (keyType) {
|
||||
case SIMPLE:
|
||||
partitionPath.append(new SimpleKeyGenerator(properties, partitionPathField).getPartitionPath(record));
|
||||
if (record.isPresent()) {
|
||||
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(record.get()));
|
||||
} else {
|
||||
partitionPath.append(new SimpleKeyGenerator(config, partitionPathField).getPartitionPath(row.get()));
|
||||
}
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
try {
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(properties, partitionPathField).getPartitionPath(record));
|
||||
if (record.isPresent()) {
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(record.get()));
|
||||
} else {
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(row.get()));
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieDeltaStreamerException("Unable to initialise TimestampBasedKeyGenerator class");
|
||||
}
|
||||
@@ -114,20 +124,23 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
if (recordKeyFields == null || recordKeyFields.isEmpty()) {
|
||||
validateRecordKeyFields();
|
||||
return getRecordKeyFields().size() == 1
|
||||
? new SimpleKeyGenerator(config).getRecordKey(record)
|
||||
: new ComplexKeyGenerator(config).getRecordKey(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(Row row) {
|
||||
validateRecordKeyFields();
|
||||
return getRecordKeyFields().size() == 1
|
||||
? new SimpleKeyGenerator(config).getRecordKey(row)
|
||||
: new ComplexKeyGenerator(config).getRecordKey(row);
|
||||
}
|
||||
|
||||
private void validateRecordKeyFields() {
|
||||
if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key in cfg");
|
||||
}
|
||||
|
||||
return recordKeyFields.size() == 1 ? new SimpleKeyGenerator(properties).getRecordKey(record) : new ComplexKeyGenerator(properties).getRecordKey(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,30 +22,28 @@ import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Key generator for deletes using global indices. Global index deletes do not require partition value
|
||||
* so this key generator avoids using partition value for generating HoodieKey.
|
||||
* Key generator for deletes using global indices. Global index deletes do not require partition value so this key generator
|
||||
* avoids using partition value for generating HoodieKey.
|
||||
*/
|
||||
public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
private static final String EMPTY_PARTITION = "";
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
|
||||
public GlobalDeleteKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.recordKeyFields = Arrays.stream(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.recordKeyFields = Arrays.asList(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordKey(record, recordKeyFields);
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields());
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -53,13 +51,19 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
|
||||
return EMPTY_PARTITION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(Row row) {
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(Row row) {
|
||||
return EMPTY_PARTITION;
|
||||
}
|
||||
}
|
||||
|
||||
110
hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
Normal file
110
hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
Normal file
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.keygen;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
public class KeyGenUtils {
|
||||
|
||||
protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
|
||||
protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
|
||||
|
||||
protected static final String DEFAULT_PARTITION_PATH = "default";
|
||||
protected static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
|
||||
public static String getRecordKey(GenericRecord record, List<String> recordKeyFields) {
|
||||
boolean keyIsNullEmpty = true;
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (String recordKeyField : recordKeyFields) {
|
||||
String recordKeyValue = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true);
|
||||
if (recordKeyValue == null) {
|
||||
recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ",");
|
||||
} else if (recordKeyValue.isEmpty()) {
|
||||
recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ",");
|
||||
} else {
|
||||
recordKey.append(recordKeyField + ":" + recordKeyValue + ",");
|
||||
keyIsNullEmpty = false;
|
||||
}
|
||||
}
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
if (keyIsNullEmpty) {
|
||||
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
|
||||
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
|
||||
}
|
||||
return recordKey.toString();
|
||||
}
|
||||
|
||||
public static String getRecordPartitionPath(GenericRecord record, List<String> partitionPathFields,
|
||||
boolean hiveStylePartitioning, boolean encodePartitionPath) {
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
String fieldVal = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
|
||||
if (fieldVal == null || fieldVal.isEmpty()) {
|
||||
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH
|
||||
: DEFAULT_PARTITION_PATH);
|
||||
} else {
|
||||
if (encodePartitionPath) {
|
||||
try {
|
||||
fieldVal = URLEncoder.encode(fieldVal, StandardCharsets.UTF_8.toString());
|
||||
} catch (UnsupportedEncodingException uoe) {
|
||||
throw new HoodieException(uoe.getMessage(), uoe);
|
||||
}
|
||||
}
|
||||
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal);
|
||||
}
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
return partitionPath.toString();
|
||||
}
|
||||
|
||||
public static String getRecordKey(GenericRecord record, String recordKeyField) {
|
||||
String recordKey = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true);
|
||||
if (recordKey == null || recordKey.isEmpty()) {
|
||||
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
|
||||
}
|
||||
return recordKey;
|
||||
}
|
||||
|
||||
public static String getPartitionPath(GenericRecord record, String partitionPathField,
|
||||
boolean hiveStylePartitioning, boolean encodePartitionPath) {
|
||||
String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
|
||||
if (partitionPath == null || partitionPath.isEmpty()) {
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
if (encodePartitionPath) {
|
||||
try {
|
||||
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
|
||||
} catch (UnsupportedEncodingException uoe) {
|
||||
throw new HoodieException(uoe.getMessage(), uoe);
|
||||
}
|
||||
}
|
||||
if (hiveStylePartitioning) {
|
||||
partitionPath = partitionPathField + "=" + partitionPath;
|
||||
}
|
||||
return partitionPath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.keygen;
|
||||
|
||||
import org.apache.hudi.AvroConversionHelper;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import scala.Function1;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
|
||||
*/
|
||||
public abstract class KeyGenerator implements Serializable, KeyGeneratorInterface {
|
||||
|
||||
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
|
||||
private static final String NAMESPACE = "hoodieRow";
|
||||
|
||||
protected transient TypedProperties config;
|
||||
private transient Function1<Object, Object> converterFn = null;
|
||||
|
||||
protected KeyGenerator(TypedProperties config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a Hoodie Key out of provided generic record.
|
||||
*/
|
||||
public abstract HoodieKey getKey(GenericRecord record);
|
||||
|
||||
/**
|
||||
* Used during bootstrap, to project out only the record key fields from bootstrap source dataset.
|
||||
*
|
||||
* @return list of field names, when concatenated make up the record key.
|
||||
*/
|
||||
public List<String> getRecordKeyFieldNames() {
|
||||
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
|
||||
+ "Please override this method in your custom key generator.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch record key from {@link Row}.
|
||||
* @param row instance of {@link Row} from which record key is requested.
|
||||
* @return the record key of interest from {@link Row}.
|
||||
*/
|
||||
public String getRecordKey(Row row) {
|
||||
if (null == converterFn) {
|
||||
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
|
||||
}
|
||||
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
|
||||
return getKey(genericRecord).getRecordKey();
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch partition path from {@link Row}.
|
||||
* @param row instance of {@link Row} from which partition path is requested
|
||||
* @return the partition path of interest from {@link Row}.
|
||||
*/
|
||||
public String getPartitionPath(Row row) {
|
||||
if (null == converterFn) {
|
||||
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
|
||||
}
|
||||
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
|
||||
return getKey(genericRecord).getPartitionPath();
|
||||
}
|
||||
}
|
||||
@@ -18,10 +18,10 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
@@ -32,12 +32,10 @@ import java.util.List;
|
||||
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
private static final String EMPTY_PARTITION = "";
|
||||
|
||||
protected final String recordKeyField;
|
||||
private static final List<String> EMPTY_PARTITION_FIELD_LIST = new ArrayList<>();
|
||||
|
||||
public NonpartitionedKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -47,6 +45,11 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return new ArrayList<>();
|
||||
return EMPTY_PARTITION_FIELD_LIST;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(Row row) {
|
||||
return EMPTY_PARTITION;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,202 @@
|
||||
/*
|
||||
* 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.keygen;
|
||||
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.types.StructField;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import scala.Option;
|
||||
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
|
||||
|
||||
/**
|
||||
* Helper class to fetch fields from Row.
|
||||
*/
|
||||
public class RowKeyGeneratorHelper {
|
||||
|
||||
/**
|
||||
* Generates record key for the corresponding {@link Row}.
|
||||
* @param row instance of {@link Row} of interest
|
||||
* @param recordKeyFields record key fields as a list
|
||||
* @param recordKeyPositions record key positions for the corresponding record keys in {@code recordKeyFields}
|
||||
* @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise.
|
||||
* @return the record key thus generated
|
||||
*/
|
||||
public static String getRecordKeyFromRow(Row row, List<String> recordKeyFields, Map<String, List<Integer>> recordKeyPositions, boolean prefixFieldName) {
|
||||
AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true);
|
||||
String toReturn = recordKeyFields.stream().map(field -> {
|
||||
String val = null;
|
||||
List<Integer> fieldPositions = recordKeyPositions.get(field);
|
||||
if (fieldPositions.size() == 1) { // simple field
|
||||
Integer fieldPos = fieldPositions.get(0);
|
||||
if (row.isNullAt(fieldPos)) {
|
||||
val = NULL_RECORDKEY_PLACEHOLDER;
|
||||
} else {
|
||||
val = row.getAs(field).toString();
|
||||
if (val.isEmpty()) {
|
||||
val = EMPTY_RECORDKEY_PLACEHOLDER;
|
||||
} else {
|
||||
keyIsNullOrEmpty.set(false);
|
||||
}
|
||||
}
|
||||
} else { // nested fields
|
||||
val = getNestedFieldVal(row, recordKeyPositions.get(field)).toString();
|
||||
if (!val.contains(NULL_RECORDKEY_PLACEHOLDER) && !val.contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
|
||||
keyIsNullOrEmpty.set(false);
|
||||
}
|
||||
}
|
||||
return prefixFieldName ? (field + ":" + val) : val;
|
||||
}).collect(Collectors.joining(","));
|
||||
if (keyIsNullOrEmpty.get()) {
|
||||
throw new HoodieKeyException("recordKey value: \"" + toReturn + "\" for fields: \"" + Arrays.toString(recordKeyFields.toArray()) + "\" cannot be null or empty.");
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates partition path for the corresponding {@link Row}.
|
||||
* @param row instance of {@link Row} of interest
|
||||
* @param partitionPathFields partition path fields as a list
|
||||
* @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise
|
||||
* @param partitionPathPositions partition path positions for the corresponding fields in {@code partitionPathFields}
|
||||
* @return the generated partition path for the row
|
||||
*/
|
||||
public static String getPartitionPathFromRow(Row row, List<String> partitionPathFields, boolean hiveStylePartitioning, Map<String, List<Integer>> partitionPathPositions) {
|
||||
return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> {
|
||||
String field = partitionPathFields.get(idx);
|
||||
String val = null;
|
||||
List<Integer> fieldPositions = partitionPathPositions.get(field);
|
||||
if (fieldPositions.size() == 1) { // simple
|
||||
Integer fieldPos = fieldPositions.get(0);
|
||||
// for partition path, if field is not found, index will be set to -1
|
||||
if (fieldPos == -1 || row.isNullAt(fieldPos)) {
|
||||
val = DEFAULT_PARTITION_PATH;
|
||||
} else {
|
||||
val = row.getAs(field).toString();
|
||||
if (val.isEmpty()) {
|
||||
val = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
}
|
||||
if (hiveStylePartitioning) {
|
||||
val = field + "=" + val;
|
||||
}
|
||||
} else { // nested
|
||||
Object nestedVal = getNestedFieldVal(row, partitionPathPositions.get(field));
|
||||
if (nestedVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER) || nestedVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
|
||||
val = hiveStylePartitioning ? field + "=" + DEFAULT_PARTITION_PATH : DEFAULT_PARTITION_PATH;
|
||||
} else {
|
||||
val = hiveStylePartitioning ? field + "=" + nestedVal.toString() : nestedVal.toString();
|
||||
}
|
||||
}
|
||||
return val;
|
||||
}).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR));
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch the field value located at the positions requested for.
|
||||
* @param row instance of {@link Row} of interest
|
||||
* @param positions tree style positions where the leaf node need to be fetched and returned
|
||||
* @return the field value as per the positions requested for.
|
||||
*/
|
||||
public static Object getNestedFieldVal(Row row, List<Integer> positions) {
|
||||
if (positions.size() == 1 && positions.get(0) == -1) {
|
||||
return DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
int index = 0;
|
||||
int totalCount = positions.size();
|
||||
Row valueToProcess = row;
|
||||
Object toReturn = null;
|
||||
|
||||
while (index < totalCount) {
|
||||
if (index < totalCount - 1) {
|
||||
if (valueToProcess.isNullAt(positions.get(index))) {
|
||||
toReturn = NULL_RECORDKEY_PLACEHOLDER;
|
||||
break;
|
||||
}
|
||||
valueToProcess = (Row) valueToProcess.get(positions.get(index));
|
||||
} else { // last index
|
||||
if (valueToProcess.getAs(positions.get(index)).toString().isEmpty()) {
|
||||
toReturn = EMPTY_RECORDKEY_PLACEHOLDER;
|
||||
break;
|
||||
}
|
||||
toReturn = valueToProcess.getAs(positions.get(index));
|
||||
}
|
||||
index++;
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate the tree style positions for the field requested for as per the defined struct type.
|
||||
* @param structType schema of interest
|
||||
* @param field field of interest for which the positions are requested for
|
||||
* @param isRecordKey {@code true} if the field requested for is a record key. {@code false} incase of a partition path.
|
||||
* @return the positions of the field as per the struct type.
|
||||
*/
|
||||
public static List<Integer> getNestedFieldIndices(StructType structType, String field, boolean isRecordKey) {
|
||||
String[] slices = field.split("\\.");
|
||||
List<Integer> positions = new ArrayList<>();
|
||||
int index = 0;
|
||||
int totalCount = slices.length;
|
||||
while (index < totalCount) {
|
||||
String slice = slices[index];
|
||||
Option<Object> curIndexOpt = structType.getFieldIndex(slice);
|
||||
if (curIndexOpt.isDefined()) {
|
||||
int curIndex = (int) curIndexOpt.get();
|
||||
positions.add(curIndex);
|
||||
final StructField nestedField = structType.fields()[curIndex];
|
||||
if (index < totalCount - 1) {
|
||||
if (!(nestedField.dataType() instanceof StructType)) {
|
||||
if (isRecordKey) {
|
||||
throw new HoodieKeyException("Nested field should be of type StructType " + nestedField);
|
||||
} else {
|
||||
positions = Collections.singletonList(-1);
|
||||
break;
|
||||
}
|
||||
}
|
||||
structType = (StructType) nestedField.dataType();
|
||||
}
|
||||
} else {
|
||||
if (isRecordKey) {
|
||||
throw new HoodieKeyException("Can't find " + slice + " in StructType for the field " + field);
|
||||
} else {
|
||||
positions = Collections.singletonList(-1);
|
||||
break;
|
||||
}
|
||||
}
|
||||
index++;
|
||||
}
|
||||
return positions;
|
||||
}
|
||||
}
|
||||
@@ -22,54 +22,52 @@ import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
*/
|
||||
public class SimpleKeyGenerator extends BuiltinKeyGenerator {
|
||||
|
||||
protected final String recordKeyField;
|
||||
|
||||
protected final String partitionPathField;
|
||||
|
||||
protected final boolean hiveStylePartitioning;
|
||||
|
||||
protected final boolean encodePartitionPath;
|
||||
|
||||
public SimpleKeyGenerator(TypedProperties props) {
|
||||
this(props, props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
|
||||
this(props, props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()),
|
||||
props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
|
||||
}
|
||||
|
||||
public SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
|
||||
SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
|
||||
this(props, null, partitionPathField);
|
||||
}
|
||||
|
||||
SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
|
||||
super(props);
|
||||
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
this.partitionPathField = partitionPathField;
|
||||
this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
|
||||
this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
|
||||
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
|
||||
this.recordKeyFields = recordKeyField == null
|
||||
? Collections.emptyList()
|
||||
: Collections.singletonList(recordKeyField);
|
||||
this.partitionPathFields = Collections.singletonList(partitionPathField);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return KeyGenUtils.getRecordKey(record, recordKeyField);
|
||||
return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(GenericRecord record) {
|
||||
return KeyGenUtils.getPartitionPath(record, partitionPathField, hiveStylePartitioning, encodePartitionPath);
|
||||
return KeyGenUtils.getPartitionPath(record, getPartitionPathFields().get(0), hiveStylePartitioning, encodePartitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getRecordKeyFields() {
|
||||
return Arrays.asList(recordKeyField);
|
||||
public String getRecordKey(Row row) {
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return Arrays.asList(partitionPathField);
|
||||
public String getPartitionPath(Row row) {
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
|
||||
hiveStylePartitioning, partitionPathPositions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,10 +25,11 @@ import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieDeltaStreamerException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.keygen.parser.HoodieDateTimeParser;
|
||||
import org.apache.hudi.keygen.parser.HoodieDateTimeParserImpl;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
@@ -39,10 +40,14 @@ import java.io.Serializable;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.util.concurrent.TimeUnit.MILLISECONDS;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
|
||||
import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
|
||||
|
||||
/**
|
||||
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
|
||||
@@ -89,11 +94,16 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
}
|
||||
|
||||
public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
|
||||
this(config, config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
|
||||
this(config, config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()),
|
||||
config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
|
||||
}
|
||||
|
||||
public TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
|
||||
super(config, partitionPathField);
|
||||
TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
|
||||
this(config, null, partitionPathField);
|
||||
}
|
||||
|
||||
TimestampBasedKeyGenerator(TypedProperties config, String recordKeyField, String partitionPathField) throws IOException {
|
||||
super(config, recordKeyField, partitionPathField);
|
||||
String dateTimeParserClass = config.getString(Config.DATE_TIME_PARSER_PROP, HoodieDateTimeParserImpl.class.getName());
|
||||
this.parser = DataSourceUtils.createDateTimeParser(config, dateTimeParserClass);
|
||||
this.outputDateTimeZone = parser.getOutputDateTimeZone();
|
||||
@@ -125,49 +135,58 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(GenericRecord record) {
|
||||
Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, partitionPathField, true);
|
||||
Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, getPartitionPathFields().get(0), true);
|
||||
if (partitionVal == null) {
|
||||
partitionVal = 1L;
|
||||
}
|
||||
try {
|
||||
return getPartitionPath(partitionVal);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse and fetch partition path based on data type.
|
||||
*
|
||||
* @param partitionVal partition path object value fetched from record/row
|
||||
* @return the parsed partition path based on data type
|
||||
* @throws ParseException on any parse exception
|
||||
*/
|
||||
private String getPartitionPath(Object partitionVal) throws ParseException {
|
||||
DateTimeFormatter partitionFormatter = DateTimeFormat.forPattern(outputDateFormat);
|
||||
if (this.outputDateTimeZone != null) {
|
||||
partitionFormatter = partitionFormatter.withZone(outputDateTimeZone);
|
||||
}
|
||||
|
||||
try {
|
||||
long timeMs;
|
||||
if (partitionVal instanceof Double) {
|
||||
timeMs = convertLongTimeToMillis(((Double) partitionVal).longValue());
|
||||
} else if (partitionVal instanceof Float) {
|
||||
timeMs = convertLongTimeToMillis(((Float) partitionVal).longValue());
|
||||
} else if (partitionVal instanceof Long) {
|
||||
timeMs = convertLongTimeToMillis((Long) partitionVal);
|
||||
} else if (partitionVal instanceof CharSequence) {
|
||||
DateTime parsedDateTime = inputFormatter.parseDateTime(partitionVal.toString());
|
||||
if (this.outputDateTimeZone == null) {
|
||||
// Use the timezone that came off the date that was passed in, if it had one
|
||||
partitionFormatter = partitionFormatter.withZone(parsedDateTime.getZone());
|
||||
}
|
||||
|
||||
timeMs = inputFormatter.parseDateTime(partitionVal.toString()).getMillis();
|
||||
} else {
|
||||
throw new HoodieNotSupportedException(
|
||||
"Unexpected type for partition field: " + partitionVal.getClass().getName());
|
||||
long timeMs;
|
||||
if (partitionVal instanceof Double) {
|
||||
timeMs = convertLongTimeToMillis(((Double) partitionVal).longValue());
|
||||
} else if (partitionVal instanceof Float) {
|
||||
timeMs = convertLongTimeToMillis(((Float) partitionVal).longValue());
|
||||
} else if (partitionVal instanceof Long) {
|
||||
timeMs = convertLongTimeToMillis((Long) partitionVal);
|
||||
} else if (partitionVal instanceof CharSequence) {
|
||||
DateTime parsedDateTime = inputFormatter.parseDateTime(partitionVal.toString());
|
||||
if (this.outputDateTimeZone == null) {
|
||||
// Use the timezone that came off the date that was passed in, if it had one
|
||||
partitionFormatter = partitionFormatter.withZone(parsedDateTime.getZone());
|
||||
}
|
||||
DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
|
||||
String partitionPath = timestamp.toString(partitionFormatter);
|
||||
if (encodePartitionPath) {
|
||||
try {
|
||||
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
|
||||
} catch (UnsupportedEncodingException uoe) {
|
||||
throw new HoodieException(uoe.getMessage(), uoe);
|
||||
}
|
||||
}
|
||||
return hiveStylePartitioning ? partitionPathField + "=" + partitionPath : partitionPath;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e);
|
||||
|
||||
timeMs = inputFormatter.parseDateTime(partitionVal.toString()).getMillis();
|
||||
} else {
|
||||
throw new HoodieNotSupportedException(
|
||||
"Unexpected type for partition field: " + partitionVal.getClass().getName());
|
||||
}
|
||||
DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
|
||||
String partitionPath = timestamp.toString(partitionFormatter);
|
||||
if (encodePartitionPath) {
|
||||
try {
|
||||
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
|
||||
} catch (UnsupportedEncodingException uoe) {
|
||||
throw new HoodieException(uoe.getMessage(), uoe);
|
||||
}
|
||||
}
|
||||
return hiveStylePartitioning ? getPartitionPathFields().get(0) + "=" + partitionPath : partitionPath;
|
||||
}
|
||||
|
||||
private long convertLongTimeToMillis(Long partitionVal) {
|
||||
@@ -177,4 +196,28 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
}
|
||||
return MILLISECONDS.convert(partitionVal, timeUnit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(Row row) {
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(Row row) {
|
||||
Object fieldVal = null;
|
||||
buildFieldPositionMapIfNeeded(row.schema());
|
||||
Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathPositions.get(getPartitionPathFields().get(0)));
|
||||
try {
|
||||
if (partitionPathFieldVal.toString().contains(DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER)
|
||||
|| partitionPathFieldVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
|
||||
fieldVal = 1L;
|
||||
} else {
|
||||
fieldVal = partitionPathFieldVal;
|
||||
}
|
||||
return getPartitionPath(fieldVal);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + fieldVal, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -228,6 +228,13 @@ object DataSourceWriteOptions {
|
||||
val KEYGENERATOR_CLASS_OPT_KEY = "hoodie.datasource.write.keygenerator.class"
|
||||
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName
|
||||
|
||||
/**
|
||||
* When set to true, will perform write operations directly using the spark native `Row` representation.
|
||||
* By default, false (will be enabled as default in a future release)
|
||||
*/
|
||||
val ENABLE_ROW_WRITER_OPT_KEY = "hoodie.datasource.write.row.writer.enable"
|
||||
val DEFAULT_ENABLE_ROW_WRITER_OPT_VAL = "false"
|
||||
|
||||
/**
|
||||
* Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata.
|
||||
* This is useful to store checkpointing information, in a consistent way with the hoodie timeline
|
||||
@@ -299,6 +306,6 @@ object DataSourceWriteOptions {
|
||||
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = "true"
|
||||
|
||||
// Async Compaction - Enabled by default for MOR
|
||||
val ASYNC_COMPACT_ENABLE_KEY = "hoodie.datasource.compaction.async.enable"
|
||||
val DEFAULT_ASYNC_COMPACT_ENABLE_VAL = "true"
|
||||
val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable"
|
||||
val DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL = "true"
|
||||
}
|
||||
|
||||
@@ -118,14 +118,12 @@ class DefaultSource extends RelationProvider
|
||||
mode: SaveMode,
|
||||
optParams: Map[String, String],
|
||||
df: DataFrame): BaseRelation = {
|
||||
val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams)
|
||||
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
|
||||
if (parameters(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
|
||||
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, df)
|
||||
} else {
|
||||
HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df)
|
||||
}
|
||||
|
||||
new HoodieEmptyRelation(sqlContext, df.schema)
|
||||
}
|
||||
|
||||
@@ -133,7 +131,7 @@ class DefaultSource extends RelationProvider
|
||||
optParams: Map[String, String],
|
||||
partitionColumns: Seq[String],
|
||||
outputMode: OutputMode): Sink = {
|
||||
val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
|
||||
new HoodieStreamingSink(
|
||||
sqlContext,
|
||||
parameters,
|
||||
|
||||
@@ -29,7 +29,6 @@ import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.client.{HoodieWriteClient, WriteStatus}
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType}
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
@@ -38,6 +37,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, B
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||
import org.apache.hudi.internal.HoodieDataSourceInternalWriter
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.SparkContext
|
||||
@@ -62,7 +62,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
asyncCompactionTriggerFn: Option[Function1[HoodieWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
|
||||
)
|
||||
: (Boolean, common.util.Option[String], common.util.Option[String],
|
||||
HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
|
||||
HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
|
||||
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val path = parameters.get("path")
|
||||
@@ -105,6 +105,22 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
} else {
|
||||
// Handle various save modes
|
||||
handleSaveModes(mode, basePath, tableConfig, tblName, operation, fs)
|
||||
// Create the table if not present
|
||||
if (!tableExists) {
|
||||
val tableMetaClient = HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get,
|
||||
HoodieTableType.valueOf(tableType), tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY),
|
||||
null.asInstanceOf[String])
|
||||
tableConfig = tableMetaClient.getTableConfig
|
||||
}
|
||||
|
||||
// short-circuit if bulk_insert via row is enabled.
|
||||
// scalastyle:off
|
||||
if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean) {
|
||||
val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName,
|
||||
basePath, path, instantTime)
|
||||
return (success, commitTime, common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig)
|
||||
}
|
||||
// scalastyle:on
|
||||
|
||||
val (writeStatuses, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) =
|
||||
if (!operation.equalsIgnoreCase(DELETE_OPERATION_OPT_VAL)) {
|
||||
@@ -128,14 +144,6 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
}).toJavaRDD()
|
||||
|
||||
// Create the table if not present
|
||||
if (!tableExists) {
|
||||
val tableMetaClient = HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get,
|
||||
HoodieTableType.valueOf(tableType), tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY),
|
||||
null.asInstanceOf[String])
|
||||
tableConfig = tableMetaClient.getTableConfig
|
||||
}
|
||||
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
|
||||
tblName, mapAsJavaMap(parameters)
|
||||
@@ -250,41 +258,29 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
metaSyncSuccess
|
||||
}
|
||||
|
||||
/**
|
||||
* Add default options for unspecified write options keys.
|
||||
*
|
||||
* @param parameters
|
||||
* @return
|
||||
*/
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL,
|
||||
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
|
||||
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
|
||||
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
|
||||
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
|
||||
COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL,
|
||||
INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL,
|
||||
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
|
||||
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
|
||||
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
|
||||
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
|
||||
//just for backwards compatiblity
|
||||
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
|
||||
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
|
||||
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
|
||||
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
|
||||
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
|
||||
HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL,
|
||||
HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL,
|
||||
HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL,
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL,
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL,
|
||||
HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL,
|
||||
ASYNC_COMPACT_ENABLE_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_VAL
|
||||
) ++ translateStorageTypeToTableType(parameters)
|
||||
def bulkInsertAsRow(sqlContext: SQLContext,
|
||||
parameters: Map[String, String],
|
||||
df: DataFrame,
|
||||
tblName: String,
|
||||
basePath: Path,
|
||||
path: Option[String],
|
||||
instantTime: String): (Boolean, common.util.Option[String]) = {
|
||||
val structName = s"${tblName}_record"
|
||||
val nameSpace = s"hoodie.${tblName}"
|
||||
val writeConfig = DataSourceUtils.createHoodieConfig(null, path.get, tblName, mapAsJavaMap(parameters))
|
||||
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
|
||||
hoodieDF.write.format("org.apache.hudi.internal")
|
||||
.option(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY, instantTime)
|
||||
.options(parameters)
|
||||
.save()
|
||||
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val syncHiveSucess = if (hiveSyncEnabled || metaSyncEnabled) {
|
||||
metaSync(parameters, basePath, sqlContext.sparkContext.hadoopConfiguration)
|
||||
} else {
|
||||
true
|
||||
}
|
||||
(syncHiveSucess, common.util.Option.ofNullable(instantTime))
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
@@ -298,7 +294,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
if (mode == SaveMode.Append && tableExists) {
|
||||
val existingTableName = tableConfig.getTableName
|
||||
if (!existingTableName.equals(tableName)) {
|
||||
throw new HoodieException(s"hoodie table with name $existingTableName already exist at $tablePath")
|
||||
throw new HoodieException(s"hoodie table with name $existingTableName already exists at $tablePath")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -411,11 +407,11 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
|
||||
val asyncCompactionEnabled = isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())
|
||||
val compactionInstant : common.util.Option[java.lang.String] =
|
||||
if (asyncCompactionEnabled) {
|
||||
client.scheduleCompaction(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
} else {
|
||||
common.util.Option.empty()
|
||||
}
|
||||
if (asyncCompactionEnabled) {
|
||||
client.scheduleCompaction(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
} else {
|
||||
common.util.Option.empty()
|
||||
}
|
||||
|
||||
log.info(s"Compaction Scheduled is $compactionInstant")
|
||||
val metaSyncSuccess = metaSync(parameters, basePath, jsc.hadoopConfiguration())
|
||||
@@ -448,7 +444,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
parameters: Map[String, String], configuration: Configuration) : Boolean = {
|
||||
log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}")
|
||||
if (!client.getConfig.isInlineCompaction
|
||||
&& parameters.get(ASYNC_COMPACT_ENABLE_KEY).exists(r => r.toBoolean)) {
|
||||
&& parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY).exists(r => r.toBoolean)) {
|
||||
tableConfig.getTableType == HoodieTableType.MERGE_ON_READ
|
||||
} else {
|
||||
false
|
||||
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* WriterUtils to assist in write path in Datasource and tests.
|
||||
*/
|
||||
object HoodieWriterUtils {
|
||||
|
||||
def javaParametersWithWriteDefaults(parameters: java.util.Map[String, String]): java.util.Map[String, String] = {
|
||||
mapAsJavaMap(parametersWithWriteDefaults(parameters.asScala.toMap))
|
||||
}
|
||||
|
||||
/**
|
||||
* Add default options for unspecified write options keys.
|
||||
*
|
||||
* @param parameters
|
||||
* @return
|
||||
*/
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL,
|
||||
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
|
||||
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
|
||||
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
|
||||
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
|
||||
COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL,
|
||||
INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL,
|
||||
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
|
||||
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
|
||||
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
|
||||
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
|
||||
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
|
||||
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
|
||||
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
|
||||
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
|
||||
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
|
||||
HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL,
|
||||
HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL,
|
||||
HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL,
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL,
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL,
|
||||
HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL,
|
||||
ASYNC_COMPACT_ENABLE_OPT_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL,
|
||||
ENABLE_ROW_WRITER_OPT_KEY -> DEFAULT_ENABLE_ROW_WRITER_OPT_VAL
|
||||
) ++ translateStorageTypeToTableType(parameters)
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
params.foreach(kv => props.setProperty(kv._1, kv._2))
|
||||
props
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user