[HUDI-838] Support schema from HoodieCommitMetadata for HiveSync (#1559)
Co-authored-by: Mehrotra <uditme@amazon.com>
This commit is contained in:
@@ -171,6 +171,16 @@ public class HoodieAvroUtils {
|
||||
return mergedSchema;
|
||||
}
|
||||
|
||||
public static Schema removeMetadataFields(Schema schema) {
|
||||
List<Schema.Field> filteredFields = schema.getFields()
|
||||
.stream()
|
||||
.filter(field -> !HoodieRecord.HOODIE_META_COLUMNS.contains(field.name()))
|
||||
.collect(Collectors.toList());
|
||||
Schema filteredSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false);
|
||||
filteredSchema.setFields(filteredFields);
|
||||
return filteredSchema;
|
||||
}
|
||||
|
||||
public static String addMetadataColumnTypes(String hiveColumnTypes) {
|
||||
return "string,string,string,string,string," + hiveColumnTypes;
|
||||
}
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.SchemaCompatibility;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
@@ -36,6 +37,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.InvalidTableException;
|
||||
@@ -66,7 +68,7 @@ public class TableSchemaResolver {
|
||||
* @return Parquet schema for this table
|
||||
* @throws Exception
|
||||
*/
|
||||
public MessageType getDataSchema() throws Exception {
|
||||
private MessageType getTableParquetSchemaFromDataFile() throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
|
||||
try {
|
||||
@@ -139,29 +141,66 @@ public class TableSchemaResolver {
|
||||
}
|
||||
}
|
||||
|
||||
private Schema getTableAvroSchemaFromDataFile() throws Exception {
|
||||
return convertParquetSchemaToAvro(getTableParquetSchemaFromDataFile());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the schema for a hoodie table in Avro format.
|
||||
* Gets full schema (user + metadata) for a hoodie table in Avro format.
|
||||
*
|
||||
* @return Avro schema for this table
|
||||
* @throws Exception
|
||||
*/
|
||||
public Schema getTableSchema() throws Exception {
|
||||
return convertParquetSchemaToAvro(getDataSchema());
|
||||
public Schema getTableAvroSchema() throws Exception {
|
||||
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(true);
|
||||
return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() : getTableAvroSchemaFromDataFile();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets full schema (user + metadata) for a hoodie table in Parquet format.
|
||||
*
|
||||
* @return Parquet schema for the table
|
||||
* @throws Exception
|
||||
*/
|
||||
public MessageType getTableParquetSchema() throws Exception {
|
||||
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(true);
|
||||
return schemaFromCommitMetadata.isPresent() ? convertAvroSchemaToParquet(schemaFromCommitMetadata.get()) :
|
||||
getTableParquetSchemaFromDataFile();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets users data schema for a hoodie table in Avro format.
|
||||
*
|
||||
* @return Avro user data schema
|
||||
* @throws Exception
|
||||
*/
|
||||
public Schema getTableAvroSchemaWithoutMetadataFields() throws Exception {
|
||||
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(false);
|
||||
return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() :
|
||||
HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the schema for a hoodie table in Avro format from the HoodieCommitMetadata of the last commit.
|
||||
*
|
||||
* @return Avro schema for this table
|
||||
* @throws Exception
|
||||
*/
|
||||
public Schema getTableSchemaFromCommitMetadata() throws Exception {
|
||||
private Option<Schema> getTableSchemaFromCommitMetadata(boolean includeMetadataFields) {
|
||||
try {
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
byte[] data = timeline.getInstantDetails(timeline.lastInstant().get()).get();
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class);
|
||||
String existingSchemaStr = metadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY);
|
||||
return new Schema.Parser().parse(existingSchemaStr);
|
||||
|
||||
if (StringUtils.isNullOrEmpty(existingSchemaStr)) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
Schema schema = new Schema.Parser().parse(existingSchemaStr);
|
||||
if (includeMetadataFields) {
|
||||
schema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
}
|
||||
return Option.of(schema);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Failed to read schema from commit metadata", e);
|
||||
}
|
||||
@@ -178,6 +217,17 @@ public class TableSchemaResolver {
|
||||
return avroSchemaConverter.convert(parquetSchema);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a avro scheme to the parquet format.
|
||||
*
|
||||
* @param schema The avro schema to convert
|
||||
* @return The converted parquet schema
|
||||
*/
|
||||
public MessageType convertAvroSchemaToParquet(Schema schema) {
|
||||
AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(metaClient.getHadoopConf());
|
||||
return avroSchemaConverter.convert(schema);
|
||||
}
|
||||
|
||||
/**
|
||||
* HUDI specific validation of schema evolution. Ensures that a newer schema can be used for the dataset by
|
||||
* checking if the data written using the old schema can be read using the new schema.
|
||||
|
||||
Reference in New Issue
Block a user