Fixing source schema and writer schema distinction in payloads
This commit is contained in:
committed by
Balaji Varadarajan
parent
395806fc68
commit
3d9041e216
@@ -31,6 +31,7 @@ import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -45,7 +46,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected final Schema schema;
|
||||
protected final Schema originalSchema;
|
||||
protected final Schema writerSchema;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
protected HoodieTimer timer;
|
||||
protected final WriteStatus writeStatus;
|
||||
@@ -56,7 +58,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
this.fs = hoodieTable.getMetaClient().getFs();
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.hoodieTimeline = hoodieTable.getCompletedCommitsTimeline();
|
||||
this.schema = createHoodieWriteSchema(config);
|
||||
this.originalSchema = new Schema.Parser().parse(config.getSchema());
|
||||
this.writerSchema = createHoodieWriteSchema(originalSchema);
|
||||
this.timer = new HoodieTimer().startTimer();
|
||||
this.writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
}
|
||||
@@ -83,8 +86,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
}
|
||||
}
|
||||
|
||||
public static Schema createHoodieWriteSchema(HoodieWriteConfig config) {
|
||||
return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
public static Schema createHoodieWriteSchema(Schema originalSchema) {
|
||||
return HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
||||
@@ -107,8 +110,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
taskAttemptId));
|
||||
}
|
||||
|
||||
public Schema getSchema() {
|
||||
return schema;
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -142,7 +145,16 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields
|
||||
* @param record
|
||||
* @return
|
||||
*/
|
||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||
return HoodieAvroUtils.rewriteRecord(record, writerSchema);
|
||||
}
|
||||
|
||||
public abstract WriteStatus close();
|
||||
|
||||
public abstract WriteStatus getWriteStatus();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user