-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[HUDI-1160] Support update partial fields for CoW table #2666
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
|
||
package org.apache.hudi.io; | ||
|
||
import org.apache.hudi.avro.HoodieAvroUtils; | ||
import org.apache.hudi.client.WriteStatus; | ||
import org.apache.hudi.common.engine.TaskContextSupplier; | ||
import org.apache.hudi.common.fs.FSUtils; | ||
|
@@ -29,10 +30,13 @@ | |
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.table.TableSchemaResolver; | ||
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.StringUtils; | ||
import org.apache.hudi.common.util.collection.ExternalSpillableMap; | ||
import org.apache.hudi.common.util.collection.Pair; | ||
import org.apache.hudi.config.HoodieWriteConfig; | ||
import org.apache.hudi.exception.HoodieCorruptedDataException; | ||
import org.apache.hudi.exception.HoodieIOException; | ||
|
@@ -106,7 +110,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H | |
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable, | ||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, | ||
TaskContextSupplier taskContextSupplier) { | ||
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); | ||
super(config, instantTime, partitionPath, fileId, hoodieTable, getWriterSchemaIncludingAndExcludingMetadataPair(config, hoodieTable), taskContextSupplier); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you help me understand something. Here we have HoodieWriteConfig at two places. As a separate entity (1st arg in constructor) and HoodieTable.getConfig(). I see within getWriterSchemaIncludingAndExcludingMetadataPair(...), we update lastSchema in hoodieWriteConfig, which will update the 1st arg. But table.getConfig() may not be updated right. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I found through debug,when config.setLastSchema is successfully updated, table.getConfig.getLastSchema is also updated, and they both take their values from the same properties |
||
init(fileId, recordItr); | ||
init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get()); | ||
} | ||
|
@@ -123,6 +127,22 @@ public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTab | |
init(fileId, this.partitionPath, dataFileToBeMerged); | ||
} | ||
|
||
protected static Pair<Schema, Schema> getWriterSchemaIncludingAndExcludingMetadataPair(HoodieWriteConfig config, HoodieTable hoodieTable) { | ||
Schema originalSchema = new Schema.Parser().parse(config.getSchema()); | ||
Schema hoodieSchema = HoodieAvroUtils.addMetadataFields(originalSchema); | ||
boolean updatePartialFields = config.updatePartialFields(); | ||
if (updatePartialFields) { | ||
try { | ||
TableSchemaResolver resolver = new TableSchemaResolver(hoodieTable.getMetaClient()); | ||
Schema lastSchema = resolver.getTableAvroSchema(); | ||
config.setLastSchema(lastSchema.toString()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. lastSchema could be null if this is first commit or last commit is an operation which does not inject schema in commit metadata. Can we check if not null and then set the last schema. If not, .toString() could throw NullPointerException. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. infact, would prefer to throw an exception if table schema is not present and if someone is trying to use updatePartialFields. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
ok There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Is this too serious? I don't think it's necessary to throw this exception here |
||
} catch (Exception e) { | ||
// Ignore exception. | ||
} | ||
} | ||
return Pair.of(originalSchema, hoodieSchema); | ||
} | ||
|
||
@Override | ||
public Schema getWriterSchemaWithMetafields() { | ||
return writerSchemaWithMetafields; | ||
|
@@ -144,6 +164,7 @@ protected String generatesDataFileName() { | |
*/ | ||
private void init(String fileId, String partitionPath, HoodieBaseFile baseFileToMerge) { | ||
LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId); | ||
writerSchemaWithMetafields = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); | ||
this.baseFileToMerge = baseFileToMerge; | ||
this.writtenRecordKeys = new HashSet<>(); | ||
writeStatus.setStat(new HoodieWriteStat()); | ||
|
@@ -266,6 +287,13 @@ protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord | |
return false; | ||
} | ||
|
||
protected GenericRecord rewriteRecord(GenericRecord record) { | ||
if (config.updatePartialFields() && !StringUtils.isNullOrEmpty(config.getLastSchema())) { | ||
return HoodieAvroUtils.rewriteRecord(record, new Schema.Parser().parse(config.getLastSchema())); | ||
} | ||
return HoodieAvroUtils.rewriteRecord(record, writerSchemaWithMetafields); | ||
} | ||
|
||
/** | ||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file. | ||
*/ | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
may be we can call this as "latest table schema".
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, i think it's good
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@danny0405 Can you help answer your questions about SQL?