[HUDI-1659] Basic Implement Of Spark Sql Support For Hoodie (#2645)
Main functions: Support create table for hoodie. Support CTAS. Support Insert for hoodie. Including dynamic partition and static partition insert. Support MergeInto for hoodie. Support DELETE Support UPDATE Both support spark2 & spark3 based on DataSourceV1. Main changes: Add sql parser for spark2. Add HoodieAnalysis for sql resolve and logical plan rewrite. Add commands implementation for CREATE TABLE、INSERT、MERGE INTO & CTAS. In order to push down the update&insert logical to the HoodieRecordPayload for MergeInto, I make same change to the HoodieWriteHandler and other related classes. 1、Add the inputSchema for parser the incoming record. This is because the inputSchema for MergeInto is different from writeSchema as there are some transforms in the update& insert expression. 2、Add WRITE_SCHEMA to HoodieWriteConfig to pass the write schema for merge into. 3、Pass properties to HoodieRecordPayload#getInsertValue to pass the insert expression and table schema. Verify this pull request Add TestCreateTable for test create hoodie tables and CTAS. Add TestInsertTable for test insert hoodie tables. Add TestMergeIntoTable for test merge hoodie tables. Add TestUpdateTable for test update hoodie tables. Add TestDeleteTable for test delete hoodie tables. Add TestSqlStatement for test supported ddl/dml currently.
This commit is contained in:
@@ -174,7 +174,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = createTable(config, hadoopConf);
|
||||
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType);
|
||||
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds,
|
||||
extraMetadata, operationType, config.getWriteSchema(), commitActionType);
|
||||
// Finalize write
|
||||
finalizeWrite(table, instantTime, stats);
|
||||
HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config);
|
||||
|
||||
@@ -164,6 +164,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
"hoodie.write.meta.key.prefixes";
|
||||
public static final String DEFAULT_WRITE_META_KEY_PREFIXES = "";
|
||||
|
||||
/**
|
||||
* The specified write schema. In most case, we do not need set this parameter,
|
||||
* but for the case the write schema is not equal to the specified table schema, we can
|
||||
* specify the write schema by this parameter.
|
||||
*
|
||||
* Currently the MergeIntoHoodieTableCommand use this to specify the write schema.
|
||||
*/
|
||||
public static final String WRITE_SCHEMA_PROP = "hoodie.write.schema";
|
||||
|
||||
/**
|
||||
* HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow
|
||||
* multiple write operations (upsert/buk-insert/...) to be executed within a single commit.
|
||||
@@ -230,6 +239,20 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
props.setProperty(AVRO_SCHEMA, schemaStr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the write schema for written records.
|
||||
*
|
||||
* If the WRITE_SCHEMA has specified, we use the WRITE_SCHEMA.
|
||||
* Or else we use the AVRO_SCHEMA as the write schema.
|
||||
* @return
|
||||
*/
|
||||
public String getWriteSchema() {
|
||||
if (props.containsKey(WRITE_SCHEMA_PROP)) {
|
||||
return props.getProperty(WRITE_SCHEMA_PROP);
|
||||
}
|
||||
return getSchema();
|
||||
}
|
||||
|
||||
public boolean getAvroSchemaValidate() {
|
||||
return Boolean.parseBoolean(props.getProperty(AVRO_SCHEMA_VALIDATE));
|
||||
}
|
||||
|
||||
@@ -18,11 +18,13 @@
|
||||
|
||||
package org.apache.hudi.execution;
|
||||
|
||||
import java.util.Properties;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.utils.LazyIterableIterator;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.CreateHandleFactory;
|
||||
@@ -82,10 +84,10 @@ public abstract class HoodieLazyInsertIterable<T extends HoodieRecordPayload>
|
||||
// It caches the exception seen while fetching insert value.
|
||||
public Option<Exception> exception = Option.empty();
|
||||
|
||||
public HoodieInsertValueGenResult(T record, Schema schema) {
|
||||
public HoodieInsertValueGenResult(T record, Schema schema, Properties properties) {
|
||||
this.record = record;
|
||||
try {
|
||||
this.insertValue = record.getData().getInsertValue(schema);
|
||||
this.insertValue = record.getData().getInsertValue(schema, properties);
|
||||
} catch (Exception e) {
|
||||
this.exception = Option.of(e);
|
||||
}
|
||||
@@ -96,9 +98,14 @@ public abstract class HoodieLazyInsertIterable<T extends HoodieRecordPayload>
|
||||
* Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
|
||||
* expensive operations of transformation to the reader thread.
|
||||
*/
|
||||
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(
|
||||
Schema schema, HoodieWriteConfig config) {
|
||||
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema, config.getProps());
|
||||
}
|
||||
|
||||
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(
|
||||
Schema schema) {
|
||||
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
|
||||
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema, CollectionUtils.EMPTY_PROPERTIES);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -189,8 +189,12 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Option<Map<String, String>> recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(writerSchema);
|
||||
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(tableSchema,
|
||||
config.getProps());
|
||||
if (avroRecord.isPresent()) {
|
||||
if (avroRecord.get().equals(IGNORE_RECORD)) {
|
||||
return avroRecord;
|
||||
}
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
|
||||
String seqId =
|
||||
@@ -336,7 +340,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
protected void appendDataAndDeleteBlocks(Map<HeaderMetadataType, String> header) {
|
||||
try {
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchemaWithMetafields.toString());
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString());
|
||||
List<HoodieLogBlock> blocks = new ArrayList<>(2);
|
||||
if (recordList.size() > 0) {
|
||||
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
|
||||
@@ -444,7 +448,10 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
}
|
||||
Option<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
if (indexedRecord.isPresent()) {
|
||||
recordList.add(indexedRecord.get());
|
||||
// Skip the Ignore Record.
|
||||
if (!indexedRecord.get().equals(IGNORE_RECORD)) {
|
||||
recordList.add(indexedRecord.get());
|
||||
}
|
||||
} else {
|
||||
keysToDelete.add(record.getKey());
|
||||
}
|
||||
|
||||
@@ -22,7 +22,7 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
@@ -38,8 +38,7 @@ public class HoodieBootstrapHandle<T extends HoodieRecordPayload, I, K, O> exten
|
||||
public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, commitTime, hoodieTable, partitionPath, fileId,
|
||||
Pair.of(HoodieAvroUtils.RECORD_KEY_SCHEMA,
|
||||
HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), taskContextSupplier);
|
||||
Option.of(HoodieAvroUtils.RECORD_KEY_SCHEMA), taskContextSupplier);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -30,7 +30,6 @@ import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
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.exception.HoodieInsertException;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||
@@ -63,14 +62,14 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config),
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, Option.empty(),
|
||||
taskContextSupplier);
|
||||
}
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||
String partitionPath, String fileId, Option<Schema> overriddenSchema,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, writerSchemaIncludingAndExcludingMetadataPair,
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, overriddenSchema,
|
||||
taskContextSupplier);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
@@ -82,7 +81,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(getPartitionId());
|
||||
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
|
||||
this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.taskContextSupplier);
|
||||
this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config,
|
||||
writeSchemaWithMetaFields, this.taskContextSupplier);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
@@ -113,6 +113,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
if (avroRecord.isPresent()) {
|
||||
if (avroRecord.get().equals(IGNORE_RECORD)) {
|
||||
return;
|
||||
}
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) avroRecord.get());
|
||||
fileWriter.writeAvroWithMetadata(recordWithMetadataInSchema, record);
|
||||
@@ -154,9 +157,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
final String key = keyIterator.next();
|
||||
HoodieRecord<T> record = recordMap.get(key);
|
||||
if (useWriterSchema) {
|
||||
write(record, record.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
write(record, record.getData().getInsertValue(tableSchemaWithMetaFields, config.getProps()));
|
||||
} else {
|
||||
write(record, record.getData().getInsertValue(writerSchema));
|
||||
write(record, record.getData().getInsertValue(tableSchema, config.getProps()));
|
||||
}
|
||||
}
|
||||
} catch (IOException io) {
|
||||
|
||||
@@ -131,12 +131,12 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getWriterSchemaWithMetafields() {
|
||||
return writerSchemaWithMetafields;
|
||||
public Schema getWriterSchemaWithMetaFields() {
|
||||
return writeSchemaWithMetaFields;
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
return writeSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -171,7 +171,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
createMarkerFile(partitionPath, newFileName);
|
||||
|
||||
// Create the writer for writing the new version file
|
||||
fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, taskContextSupplier);
|
||||
fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config,
|
||||
writeSchemaWithMetaFields, taskContextSupplier);
|
||||
} catch (IOException io) {
|
||||
LOG.error("Error in update task at commit " + instantTime, io);
|
||||
writeStatus.setGlobalError(io);
|
||||
@@ -198,7 +199,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps());
|
||||
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema));
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
@@ -284,11 +285,17 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
||||
try {
|
||||
Option<IndexedRecord> combinedAvroRecord =
|
||||
hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchemaWithMetafields : writerSchema,
|
||||
hoodieRecord.getData().combineAndGetUpdateValue(oldRecord,
|
||||
useWriterSchema ? tableSchemaWithMetaFields : tableSchema,
|
||||
config.getPayloadConfig().getProps());
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
|
||||
if (combinedAvroRecord.isPresent() && combinedAvroRecord.get().equals(IGNORE_RECORD)) {
|
||||
// If it is an IGNORE_RECORD, just copy the old record, and do not update the new record.
|
||||
copyOldRecord = true;
|
||||
} else if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/*
|
||||
* ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully write the the combined new
|
||||
* ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully
|
||||
* write the the combined new
|
||||
* value
|
||||
*
|
||||
* We no longer need to copy the old record over.
|
||||
@@ -308,7 +315,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
fileWriter.writeAvro(key, oldRecord);
|
||||
} catch (IOException | RuntimeException e) {
|
||||
String errMsg = String.format("Failed to merge old record into new file for key %s from old file %s to new file %s with writerSchema %s",
|
||||
key, getOldFilePath(), newFilePath, writerSchemaWithMetafields.toString(true));
|
||||
key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true));
|
||||
LOG.debug("Old record is " + oldRecord);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
@@ -325,11 +332,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> hoodieRecord = newRecordsItr.next();
|
||||
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
Schema schema = useWriterSchema ? tableSchemaWithMetaFields : tableSchema;
|
||||
Option<IndexedRecord> insertRecord =
|
||||
hoodieRecord.getData().getInsertValue(schema, config.getProps());
|
||||
// just skip the ignore record
|
||||
if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) {
|
||||
continue;
|
||||
}
|
||||
writeRecord(hoodieRecord, insertRecord);
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -87,9 +87,9 @@ public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> ext
|
||||
}
|
||||
try {
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchemaWithMetaFields));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
writtenRecordKeys.add(keyToPreWrite);
|
||||
@@ -109,9 +109,9 @@ public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> ext
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchemaWithMetaFields));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
|
||||
@@ -28,7 +28,6 @@ import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||
@@ -55,8 +54,44 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class);
|
||||
|
||||
protected final Schema writerSchema;
|
||||
protected final Schema writerSchemaWithMetafields;
|
||||
/**
|
||||
* A special record returned by {@link HoodieRecordPayload}, which means
|
||||
* {@link HoodieWriteHandle} should just skip this record.
|
||||
* This record is only used for {@link HoodieRecordPayload} currently, so it should not
|
||||
* shuffle though network, we can compare the record locally by the equal method.
|
||||
* The HoodieRecordPayload#combineAndGetUpdateValue and HoodieRecordPayload#getInsertValue
|
||||
* have 3 kind of return:
|
||||
* 1、Option.empty
|
||||
* This means we should delete this record.
|
||||
* 2、IGNORE_RECORD
|
||||
* This means we should not process this record,just skip.
|
||||
* 3、Other non-empty record
|
||||
* This means we should process this record.
|
||||
*
|
||||
* We can see the usage of IGNORE_RECORD in
|
||||
* org.apache.spark.sql.hudi.command.payload.ExpressionPayload
|
||||
*/
|
||||
public static IgnoreRecord IGNORE_RECORD = new IgnoreRecord();
|
||||
|
||||
/**
|
||||
* The specified schema of the table. ("specified" denotes that this is configured by the client,
|
||||
* as opposed to being implicitly fetched out of the commit metadata)
|
||||
*/
|
||||
protected final Schema tableSchema;
|
||||
protected final Schema tableSchemaWithMetaFields;
|
||||
|
||||
/**
|
||||
* The write schema. In most case the write schema is the same to the
|
||||
* input schema. But if HoodieWriteConfig#WRITE_SCHEMA is specified,
|
||||
* we use the WRITE_SCHEMA as the write schema.
|
||||
*
|
||||
* This is useful for the case of custom HoodieRecordPayload which do some conversion
|
||||
* to the incoming record in it. e.g. the ExpressionPayload do the sql expression conversion
|
||||
* to the input.
|
||||
*/
|
||||
protected final Schema writeSchema;
|
||||
protected final Schema writeSchemaWithMetaFields;
|
||||
|
||||
protected HoodieTimer timer;
|
||||
protected WriteStatus writeStatus;
|
||||
protected final String partitionPath;
|
||||
@@ -67,17 +102,19 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
|
||||
String fileId, HoodieTable<T, I, K, O> hoodieTable, TaskContextSupplier taskContextSupplier) {
|
||||
this(config, instantTime, partitionPath, fileId, hoodieTable,
|
||||
getWriterSchemaIncludingAndExcludingMetadataPair(config), taskContextSupplier);
|
||||
Option.empty(), taskContextSupplier);
|
||||
}
|
||||
|
||||
protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId,
|
||||
HoodieTable<T, I, K, O> hoodieTable, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||
HoodieTable<T, I, K, O> hoodieTable, Option<Schema> overriddenSchema,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable);
|
||||
this.partitionPath = partitionPath;
|
||||
this.fileId = fileId;
|
||||
this.writerSchema = writerSchemaIncludingAndExcludingMetadataPair.getKey();
|
||||
this.writerSchemaWithMetafields = writerSchemaIncludingAndExcludingMetadataPair.getValue();
|
||||
this.tableSchema = overriddenSchema.orElseGet(() -> getSpecifiedTableSchema(config));
|
||||
this.tableSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(tableSchema);
|
||||
this.writeSchema = overriddenSchema.orElseGet(() -> getWriteSchema(config));
|
||||
this.writeSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(writeSchema);
|
||||
this.timer = new HoodieTimer().startTimer();
|
||||
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
|
||||
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
|
||||
@@ -86,16 +123,22 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns writer schema pairs containing
|
||||
* (a) Writer Schema from client
|
||||
* (b) (a) with hoodie metadata fields.
|
||||
* @param config Write Config
|
||||
* Get the specified table schema.
|
||||
* @param config
|
||||
* @return
|
||||
*/
|
||||
protected static Pair<Schema, Schema> getWriterSchemaIncludingAndExcludingMetadataPair(HoodieWriteConfig config) {
|
||||
Schema originalSchema = new Schema.Parser().parse(config.getSchema());
|
||||
Schema hoodieSchema = HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
return Pair.of(originalSchema, hoodieSchema);
|
||||
private static Schema getSpecifiedTableSchema(HoodieWriteConfig config) {
|
||||
return new Schema.Parser().parse(config.getSchema());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the schema, of the actual write.
|
||||
*
|
||||
* @param config
|
||||
* @return
|
||||
*/
|
||||
private static Schema getWriteSchema(HoodieWriteConfig config) {
|
||||
return new Schema.Parser().parse(config.getWriteSchema());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -138,8 +181,8 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
markerFiles.create(partitionPath, dataFileName, getIOType());
|
||||
}
|
||||
|
||||
public Schema getWriterSchemaWithMetafields() {
|
||||
return writerSchemaWithMetafields;
|
||||
public Schema getWriterSchemaWithMetaFields() {
|
||||
return writeSchemaWithMetaFields;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -177,7 +220,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
|
||||
*/
|
||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||
return HoodieAvroUtils.rewriteRecord(record, writerSchemaWithMetafields);
|
||||
return HoodieAvroUtils.rewriteRecord(record, writeSchemaWithMetaFields);
|
||||
}
|
||||
|
||||
public abstract List<WriteStatus> close();
|
||||
@@ -213,4 +256,32 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, taskContextSupplier);
|
||||
}
|
||||
|
||||
private static class IgnoreRecord implements GenericRecord {
|
||||
|
||||
@Override
|
||||
public void put(int i, Object v) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get(int i) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(String key, Object v) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get(String key) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -85,7 +85,7 @@ public class HoodieConcatHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
fileWriter.writeAvro(key, oldRecord);
|
||||
} catch (IOException | RuntimeException e) {
|
||||
String errMsg = String.format("Failed to write old record into new file for key %s from old file %s to new file %s with writerSchema %s",
|
||||
key, getOldFilePath(), newFilePath, writerSchemaWithMetafields.toString(true));
|
||||
key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true));
|
||||
LOG.debug("Old record is " + oldRecord);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
|
||||
@@ -21,16 +21,13 @@ 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.util.PartitionPathEncodeUtils;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
@@ -40,7 +37,7 @@ public class KeyGenUtils {
|
||||
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 final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
|
||||
/**
|
||||
* Extracts the record key fields in strings out of the given record key,
|
||||
@@ -104,11 +101,7 @@ public class KeyGenUtils {
|
||||
: DEFAULT_PARTITION_PATH);
|
||||
} else {
|
||||
if (encodePartitionPath) {
|
||||
try {
|
||||
fieldVal = URLEncoder.encode(fieldVal, StandardCharsets.UTF_8.toString());
|
||||
} catch (UnsupportedEncodingException uoe) {
|
||||
throw new HoodieException(uoe.getMessage(), uoe);
|
||||
}
|
||||
fieldVal = PartitionPathEncodeUtils.escapePathName(fieldVal);
|
||||
}
|
||||
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal);
|
||||
}
|
||||
@@ -133,11 +126,7 @@ public class KeyGenUtils {
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
if (encodePartitionPath) {
|
||||
try {
|
||||
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
|
||||
} catch (UnsupportedEncodingException uoe) {
|
||||
throw new HoodieException(uoe.getMessage(), uoe);
|
||||
}
|
||||
partitionPath = PartitionPathEncodeUtils.escapePathName(partitionPath);
|
||||
}
|
||||
if (hiveStylePartitioning) {
|
||||
partitionPath = partitionPathField + "=" + partitionPath;
|
||||
|
||||
@@ -21,6 +21,7 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieKeyGeneratorException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
@@ -34,10 +35,7 @@ import org.joda.time.format.DateTimeFormatter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.math.BigDecimal;
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@@ -215,11 +213,7 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
|
||||
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);
|
||||
}
|
||||
partitionPath = PartitionPathEncodeUtils.escapePathName(partitionPath);
|
||||
}
|
||||
return hiveStylePartitioning ? getPartitionPathFields().get(0) + "=" + partitionPath : partitionPath;
|
||||
}
|
||||
|
||||
@@ -39,7 +39,7 @@ public class BootstrapRecordConsumer extends BoundedInMemoryQueueConsumer<Hoodie
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieRecord record) {
|
||||
try {
|
||||
bootstrapHandle.write(record, record.getData().getInsertValue(bootstrapHandle.getWriterSchemaWithMetafields()));
|
||||
bootstrapHandle.write(record, record.getData().getInsertValue(bootstrapHandle.getWriterSchemaWithMetaFields()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
|
||||
@@ -103,7 +103,7 @@ public abstract class AbstractMergeHelper<T extends HoodieRecordPayload, I, K, O
|
||||
}
|
||||
|
||||
return new MergingIterator<>(reader.getRecordIterator(readSchema), bootstrapReader.getRecordIterator(bootstrapReadSchema),
|
||||
(inputRecordPair) -> HoodieAvroUtils.stitchRecords(inputRecordPair.getLeft(), inputRecordPair.getRight(), mergeHandle.getWriterSchemaWithMetafields()));
|
||||
(inputRecordPair) -> HoodieAvroUtils.stitchRecords(inputRecordPair.getLeft(), inputRecordPair.getRight(), mergeHandle.getWriterSchemaWithMetaFields()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user