[HUDI-2176, 2178, 2179] Adding virtual key support to COW table (#3306)
This commit is contained in:
committed by
GitHub
parent
5353243449
commit
61148c1c43
@@ -1594,6 +1594,11 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPopulateMetaFields(boolean populateMetaFields) {
|
||||
writeConfig.setValue(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withProperties(Properties properties) {
|
||||
this.writeConfig.getProps().putAll(properties);
|
||||
return this;
|
||||
|
||||
@@ -207,9 +207,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
|
||||
String seqId =
|
||||
HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
|
||||
if (config.populateMetaFields()) {
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
|
||||
}
|
||||
if (isUpdateRecord(hoodieRecord)) {
|
||||
updatedRecordsWritten++;
|
||||
} else {
|
||||
|
||||
@@ -23,12 +23,16 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -39,17 +43,25 @@ import java.util.stream.Stream;
|
||||
public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
|
||||
|
||||
private final Pair<String, HoodieBaseFile> partitionPathBaseFilePair;
|
||||
private final Option<BaseKeyGenerator> keyGeneratorOpt;
|
||||
|
||||
public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Pair<String, HoodieBaseFile> partitionPathBaseFilePair) {
|
||||
Pair<String, HoodieBaseFile> partitionPathBaseFilePair, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId()));
|
||||
this.partitionPathBaseFilePair = partitionPathBaseFilePair;
|
||||
this.keyGeneratorOpt = keyGeneratorOpt;
|
||||
}
|
||||
|
||||
public Stream<Pair<HoodieKey, HoodieRecordLocation>> locations() {
|
||||
HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight();
|
||||
return BaseFileUtils.getInstance(baseFile.getPath()).fetchRecordKeyPartitionPath(
|
||||
hoodieTable.getHadoopConf(), new Path(baseFile.getPath())).stream()
|
||||
BaseFileUtils baseFileUtils = BaseFileUtils.getInstance(baseFile.getPath());
|
||||
List<HoodieKey> hoodieKeyList = new ArrayList<>();
|
||||
if (keyGeneratorOpt.isPresent()) {
|
||||
hoodieKeyList = baseFileUtils.fetchRecordKeyPartitionPath(hoodieTable.getHadoopConf(), new Path(baseFile.getPath()), keyGeneratorOpt);
|
||||
} else {
|
||||
hoodieKeyList = baseFileUtils.fetchRecordKeyPartitionPath(hoodieTable.getHadoopConf(), new Path(baseFile.getPath()));
|
||||
}
|
||||
return hoodieKeyList.stream()
|
||||
.map(entry -> Pair.of(entry,
|
||||
new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId())));
|
||||
}
|
||||
|
||||
@@ -32,6 +32,7 @@ import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCorruptedDataException;
|
||||
@@ -40,6 +41,8 @@ import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
@@ -101,21 +104,23 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
protected long updatedRecordsWritten = 0;
|
||||
protected long insertRecordsWritten = 0;
|
||||
protected boolean useWriterSchema;
|
||||
protected Option<BaseKeyGenerator> keyGeneratorOpt;
|
||||
private HoodieBaseFile baseFileToMerge;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
this(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier,
|
||||
hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get());
|
||||
hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get(), keyGeneratorOpt);
|
||||
}
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId,
|
||||
TaskContextSupplier taskContextSupplier, HoodieBaseFile baseFile) {
|
||||
TaskContextSupplier taskContextSupplier, HoodieBaseFile baseFile, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
|
||||
init(fileId, recordItr);
|
||||
init(fileId, partitionPath, baseFile);
|
||||
validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -123,11 +128,17 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
*/
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
this.useWriterSchema = true;
|
||||
init(fileId, this.partitionPath, dataFileToBeMerged);
|
||||
validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
|
||||
}
|
||||
|
||||
private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator> keyGeneratorOpt, boolean populateMetaFields) {
|
||||
ValidationUtils.checkArgument(populateMetaFields == !keyGeneratorOpt.isPresent());
|
||||
this.keyGeneratorOpt = keyGeneratorOpt;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -278,7 +289,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt);
|
||||
boolean copyOldRecord = true;
|
||||
if (keyToNewRecords.containsKey(key)) {
|
||||
// If we have duplicate records that we are updating, then the hoodie record will be deflated after
|
||||
|
||||
@@ -23,8 +23,10 @@ import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -47,8 +49,9 @@ public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> ext
|
||||
private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
|
||||
|
||||
public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier,
|
||||
Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt);
|
||||
newRecordKeysSorted.addAll(keyToNewRecords.keySet());
|
||||
}
|
||||
|
||||
@@ -57,9 +60,9 @@ public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> ext
|
||||
*/
|
||||
public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
|
||||
taskContextSupplier);
|
||||
taskContextSupplier, keyGeneratorOpt);
|
||||
|
||||
newRecordKeysSorted.addAll(keyToNewRecords.keySet());
|
||||
}
|
||||
|
||||
@@ -20,11 +20,13 @@ package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -66,13 +68,14 @@ public class HoodieConcatHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieConcatHandle.class);
|
||||
|
||||
public HoodieConcatHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator recordItr,
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt);
|
||||
}
|
||||
|
||||
public HoodieConcatHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Map keyToNewRecords, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, keyToNewRecords, partitionPath, fileId, dataFileToBeMerged, taskContextSupplier);
|
||||
super(config, instantTime, hoodieTable, keyToNewRecords, partitionPath, fileId, dataFileToBeMerged, taskContextSupplier,
|
||||
Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -80,7 +83,7 @@ public class HoodieConcatHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
*/
|
||||
@Override
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt);
|
||||
try {
|
||||
fileWriter.writeAvro(key, oldRecord);
|
||||
} catch (IOException | RuntimeException e) {
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
@@ -34,9 +35,9 @@ import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
|
||||
|
||||
public class HoodieFileWriterFactory {
|
||||
|
||||
@@ -45,7 +46,7 @@ public class HoodieFileWriterFactory {
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
final String extension = FSUtils.getFileExtension(path.getName());
|
||||
if (PARQUET.getFileExtension().equals(extension)) {
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, config.populateMetaFields());
|
||||
}
|
||||
if (HFILE.getFileExtension().equals(extension)) {
|
||||
return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
|
||||
@@ -58,16 +59,21 @@ public class HoodieFileWriterFactory {
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
BloomFilter filter = createBloomFilter(config);
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter(hoodieTable.getHadoopConf()).convert(schema), schema, filter);
|
||||
TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException {
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, populateMetaFields, populateMetaFields);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
TaskContextSupplier taskContextSupplier, boolean populateMetaFields, boolean enableBloomFilter) throws IOException {
|
||||
Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(hoodieTable.getHadoopConf()).convert(schema), schema, filter);
|
||||
|
||||
HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(),
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
|
||||
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio());
|
||||
|
||||
return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier);
|
||||
return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier, populateMetaFields);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newHFileFileWriter(
|
||||
|
||||
@@ -49,9 +49,10 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String instantTime;
|
||||
private final TaskContextSupplier taskContextSupplier;
|
||||
private final boolean populateMetaFields;
|
||||
|
||||
public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig,
|
||||
Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
Schema schema, TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
|
||||
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
@@ -69,14 +70,19 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.instantTime = instantTime;
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
this.populateMetaFields = populateMetaFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
prepRecordWithMetadata(avroRecord, record, instantTime,
|
||||
taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, file.getName());
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
if (populateMetaFields) {
|
||||
prepRecordWithMetadata(avroRecord, record, instantTime,
|
||||
taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, file.getName());
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
} else {
|
||||
super.write(avroRecord);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -87,7 +93,9 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
@Override
|
||||
public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||
super.write(object);
|
||||
writeSupport.add(key);
|
||||
if (populateMetaFields) {
|
||||
writeSupport.add(key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -1,81 +0,0 @@
|
||||
/*
|
||||
* 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.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class BaseKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected List<String> recordKeyFields;
|
||||
protected List<String> partitionPathFields;
|
||||
protected final boolean encodePartitionPath;
|
||||
protected final boolean hiveStylePartitioning;
|
||||
|
||||
protected BaseKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
@Override
|
||||
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());
|
||||
}
|
||||
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
}
|
||||
}
|
||||
@@ -21,6 +21,8 @@ package org.apache.hudi.keygen;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
@@ -41,6 +43,26 @@ public class KeyGenUtils {
|
||||
protected static final String DEFAULT_PARTITION_PATH = "default";
|
||||
public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
|
||||
/**
|
||||
* Fetches record key from the GenericRecord.
|
||||
* @param genericRecord generic record of interest.
|
||||
* @param keyGeneratorOpt Optional BaseKeyGenerator. If not, meta field will be used.
|
||||
* @return the record key for the passed in generic record.
|
||||
*/
|
||||
public static String getRecordKeyFromGenericRecord(GenericRecord genericRecord, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(genericRecord) : genericRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches partition path from the GenericRecord.
|
||||
* @param genericRecord generic record of interest.
|
||||
* @param keyGeneratorOpt Optional BaseKeyGenerator. If not, meta field will be used.
|
||||
* @return the partition path for the passed in generic record.
|
||||
*/
|
||||
public static String getPartitionPathFromGenericRecord(GenericRecord genericRecord, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(genericRecord) : genericRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the record key fields in strings out of the given record key,
|
||||
* this is the reverse operation of {@link #getRecordKey(GenericRecord, String)}.
|
||||
|
||||
@@ -1,59 +0,0 @@
|
||||
/*
|
||||
* 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.ApiMaturityLevel;
|
||||
import org.apache.hudi.PublicAPIClass;
|
||||
import org.apache.hudi.PublicAPIMethod;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract class KeyGenerator implements KeyGeneratorInterface {
|
||||
|
||||
protected TypedProperties config;
|
||||
|
||||
protected KeyGenerator(TypedProperties config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a Hoodie Key out of provided generic record.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
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.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public List<String> getRecordKeyFieldNames() {
|
||||
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
|
||||
+ "Please override this method in your custom key generator.");
|
||||
}
|
||||
}
|
||||
@@ -1,36 +0,0 @@
|
||||
/*
|
||||
* 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.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents the interface key generators need to adhere to.
|
||||
*/
|
||||
public interface KeyGeneratorInterface extends Serializable {
|
||||
|
||||
HoodieKey getKey(GenericRecord record);
|
||||
|
||||
List<String> getRecordKeyFieldNames();
|
||||
|
||||
}
|
||||
@@ -1,59 +0,0 @@
|
||||
/*
|
||||
* 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.constant;
|
||||
|
||||
import org.apache.hudi.common.config.ConfigClassProperty;
|
||||
import org.apache.hudi.common.config.ConfigGroups;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
@ConfigClassProperty(name = "Key Generator Options",
|
||||
groupName = ConfigGroups.Names.WRITE_CLIENT,
|
||||
description = "Hudi maintains keys (record key + partition path) "
|
||||
+ "for uniquely identifying a particular record. "
|
||||
+ "This config allows developers to setup the Key generator class that "
|
||||
+ "will extract these out of incoming records.")
|
||||
public class KeyGeneratorOptions extends HoodieConfig {
|
||||
|
||||
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.partitionpath.urlencode")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Should we url encode the partition path value, before creating the folder structure.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.hive_style_partitioning")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Flag to indicate whether to use Hive style partitioning.\n"
|
||||
+ "If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.\n"
|
||||
+ "By default false (the names of partition folders are only partition values)");
|
||||
|
||||
public static final ConfigProperty<String> RECORDKEY_FIELD_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.recordkey.field")
|
||||
.defaultValue("uuid")
|
||||
.withDocumentation("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
|
||||
+ "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using\n"
|
||||
+ "the dot notation eg: `a.b.c`");
|
||||
|
||||
public static final ConfigProperty<String> PARTITIONPATH_FIELD_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.partitionpath.field")
|
||||
.defaultValue("partitionpath")
|
||||
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
|
||||
+ "Actual value ontained by invoking .toString()");
|
||||
}
|
||||
|
||||
@@ -1,70 +0,0 @@
|
||||
/*
|
||||
* 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.constant;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Types of {@link org.apache.hudi.keygen.KeyGenerator}.
|
||||
*/
|
||||
public enum KeyGeneratorType {
|
||||
/**
|
||||
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
*/
|
||||
SIMPLE,
|
||||
|
||||
/**
|
||||
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
|
||||
*/
|
||||
COMPLEX,
|
||||
|
||||
/**
|
||||
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
|
||||
*/
|
||||
TIMESTAMP,
|
||||
|
||||
/**
|
||||
* This is a generic implementation type 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.
|
||||
* <p>
|
||||
* This KeyGenerator 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").
|
||||
*/
|
||||
CUSTOM,
|
||||
|
||||
/**
|
||||
* Simple Key generator for unpartitioned Hive Tables.
|
||||
*/
|
||||
NON_PARTITION,
|
||||
|
||||
/**
|
||||
* Key generator for deletes using global indices.
|
||||
*/
|
||||
GLOBAL_DELETE;
|
||||
|
||||
public static List<String> getNames() {
|
||||
List<String> names = new ArrayList<>(KeyGeneratorType.values().length);
|
||||
Arrays.stream(KeyGeneratorType.values())
|
||||
.forEach(x -> names.add(x.name()));
|
||||
return names;
|
||||
}
|
||||
}
|
||||
@@ -34,6 +34,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
|
||||
import org.apache.hudi.io.storage.HoodieOrcConfig;
|
||||
@@ -68,11 +69,13 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
|
||||
|
||||
protected final Schema schema;
|
||||
protected final BloomFilter filter;
|
||||
protected final boolean populateMetaFields;
|
||||
|
||||
protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
|
||||
super(basePath, fs, metaClient);
|
||||
this.schema = schema;
|
||||
this.filter = filter;
|
||||
this.populateMetaFields = metaClient.getTableConfig().populateMetaFields();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -91,21 +94,25 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
|
||||
|
||||
if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.PARQUET)) {
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
|
||||
HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));
|
||||
try (HoodieParquetWriter writer = new HoodieParquetWriter(
|
||||
currentInstantTime,
|
||||
new Path(Paths.get(basePath, partition, fileName).toString()),
|
||||
config, schema, contextSupplier)) {
|
||||
config, schema, contextSupplier, populateMetaFields)) {
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++));
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
filter.add(record.getRecordKey());
|
||||
if (populateMetaFields) {
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++));
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
filter.add(record.getRecordKey());
|
||||
} else {
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.ORC)) {
|
||||
|
||||
Reference in New Issue
Block a user