[HUDI-1129] Improving schema evolution support in hudi (#2927)
* Adding support to ingest records with old schema after table's schema is evolved * Rebasing against latest master - Trimming test file to be < 800 lines - Renaming config names * Addressing feedback Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
73d898322b
commit
1196736185
@@ -109,6 +109,16 @@ public class FSUtils {
|
||||
return getFs(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if table already exists in the given path.
|
||||
* @param path base path of the table.
|
||||
* @param fs instance of {@link FileSystem}.
|
||||
* @return {@code true} if table exists. {@code false} otherwise.
|
||||
*/
|
||||
public static boolean isTableExists(String path, FileSystem fs) throws IOException {
|
||||
return fs.exists(new Path(path + "/" + HoodieTableMetaClient.METAFOLDER_NAME));
|
||||
}
|
||||
|
||||
public static Path addSchemeIfLocalPath(String path) {
|
||||
Path providedPath = new Path(path);
|
||||
File localFile = new File(path);
|
||||
|
||||
@@ -414,6 +414,13 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return fs.listStatus(metaPath, nameFilter);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return {@code true} if any commits are found, else {@code false}.
|
||||
*/
|
||||
public boolean isTimelineNonEmpty() {
|
||||
return getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toList()).size() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the commit timeline visible for this table.
|
||||
*/
|
||||
|
||||
@@ -18,13 +18,6 @@
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
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;
|
||||
@@ -36,11 +29,18 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
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.Functions.Function1;
|
||||
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;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
@@ -49,6 +49,8 @@ import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Helper class to read schema from data files and log files and to convert it between different formats.
|
||||
*/
|
||||
@@ -381,6 +383,37 @@ public class TableSchemaResolver {
|
||||
return isSchemaCompatible(new Schema.Parser().parse(oldSchema), new Schema.Parser().parse(newSchema));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get latest schema either from incoming schema or table schema.
|
||||
* @param writeSchema incoming batch's write schema.
|
||||
* @param convertTableSchemaToAddNamespace {@code true} if table schema needs to be converted. {@code false} otherwise.
|
||||
* @param converterFn converter function to be called over table schema (to add namespace may be). Each caller can decide if any conversion is required.
|
||||
* @return the latest schema.
|
||||
*/
|
||||
public Schema getLatestSchema(Schema writeSchema, boolean convertTableSchemaToAddNamespace,
|
||||
Function1<Schema, Schema> converterFn) {
|
||||
Schema latestSchema = writeSchema;
|
||||
try {
|
||||
if (metaClient.isTimelineNonEmpty()) {
|
||||
Schema tableSchema = getTableAvroSchemaWithoutMetadataFields();
|
||||
if (convertTableSchemaToAddNamespace && converterFn != null) {
|
||||
tableSchema = converterFn.apply(tableSchema);
|
||||
}
|
||||
if (writeSchema.getFields().size() < tableSchema.getFields().size() && isSchemaCompatible(writeSchema, tableSchema)) {
|
||||
// if incoming schema is a subset (old schema) compared to table schema. For eg, one of the
|
||||
// ingestion pipeline is still producing events in old schema
|
||||
latestSchema = tableSchema;
|
||||
LOG.debug("Using latest table schema to rewrite incoming records " + tableSchema.toString());
|
||||
}
|
||||
}
|
||||
} catch (IllegalArgumentException | InvalidTableException e) {
|
||||
LOG.warn("Could not find any commits, falling back to using incoming batch's write schema");
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Unknown exception thrown " + e.getMessage() + ", Falling back to using incoming batch's write schema");
|
||||
}
|
||||
return latestSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the parquet schema from a parquet File.
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user