1
0

[HUDI-741] Added checks to validate Hoodie's schema evolution.

HUDI specific validation of schema evolution should ensure that a newer schema can be used for the dataset by checking that the data written using the old schema can be read using the new schema.

Code changes:

1. Added a new config in HoodieWriteConfig to enable schema validation check (disabled by default)
2. Moved code that reads schema from base/log files into hudi-common from hudi-hive-sync
3. Added writerSchema to the extraMetadata of compaction commits in MOR table. This is same as that for commits on COW table.

Testing changes:

4. Extended TestHoodieClientBase to add insertBatch API which allows inserting a new batch of unique records into a HUDI table
5. Added a unit test to verify schema evolution for both COW and MOR tables.
6. Added unit tests for schema compatiblity checks.
This commit is contained in:
Prashant Wason
2020-03-27 00:53:49 -07:00
committed by n3nash
parent 9ca710cb02
commit 19d29ac7d0
15 changed files with 1004 additions and 180 deletions

View File

@@ -19,6 +19,8 @@
package org.apache.hudi.client;
import com.codahale.metrics.Timer;
import org.apache.avro.Schema;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
@@ -36,6 +38,7 @@ import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -50,9 +53,11 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieCompactionException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.exception.HoodieRestoreException;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.metrics.HoodieMetrics;
import org.apache.hudi.table.HoodieCommitArchiveLog;
@@ -166,6 +171,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT);
validateSchema(table, true);
setOperationType(WriteOperationType.UPSERT);
HoodieWriteMetadata result = table.upsert(jsc,instantTime, records);
if (result.getIndexLookupDuration().isPresent()) {
@@ -185,6 +191,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/
public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime) {
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED);
validateSchema(table, true);
setOperationType(WriteOperationType.UPSERT_PREPPED);
HoodieWriteMetadata result = table.upsertPrepped(jsc,instantTime, preppedRecords);
return postWrite(result, instantTime, table);
@@ -202,6 +209,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT);
validateSchema(table, false);
setOperationType(WriteOperationType.INSERT);
HoodieWriteMetadata result = table.insert(jsc,instantTime, records);
return postWrite(result, instantTime, table);
@@ -220,6 +228,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/
public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime) {
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED);
validateSchema(table, false);
setOperationType(WriteOperationType.INSERT_PREPPED);
HoodieWriteMetadata result = table.insertPrepped(jsc,instantTime, preppedRecords);
return postWrite(result, instantTime, table);
@@ -882,6 +891,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
metadata.addWriteStat(stat.getPartitionPath(), stat);
}
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
// Finalize write
finalizeWrite(table, compactionCommitTime, updateStatusMap);
@@ -919,4 +930,55 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
});
return compactionInstantTimeOpt;
}
}
/**
* Ensure that the current writerSchema is compatible with the latest schema of this dataset.
*
* When inserting/updating data, we read records using the last used schema and convert them to the
* GenericRecords with writerSchema. Hence, we need to ensure that this conversion can take place without errors.
*
* @param hoodieTable The Hoodie Table
* @param isUpsert If this is a check during upserts
* @throws HoodieUpsertException If schema check fails during upserts
* @throws HoodieInsertException If schema check fails during inserts
*/
private void validateSchema(HoodieTable<T> hoodieTable, final boolean isUpsert)
throws HoodieUpsertException, HoodieInsertException {
if (!getConfig().getAvroSchemaValidate()) {
// Check not required
return;
}
boolean isValid = false;
String errorMsg = "WriterSchema is not compatible with the schema present in the Table";
Throwable internalError = null;
Schema tableSchema = null;
Schema writerSchema = null;
try {
TableSchemaResolver schemaUtil = new TableSchemaResolver(hoodieTable.getMetaClient());
writerSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema());
tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableSchemaFromCommitMetadata());
isValid = schemaUtil.isSchemaCompatible(tableSchema, writerSchema);
} catch (Exception e) {
// Two error cases are possible:
// 1. There was no schema as no data has been inserted yet (first time only)
// 2. Failure in reading the schema
isValid = hoodieTable.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0;
errorMsg = "Failed to read latest schema on path " + basePath;
internalError = e;
}
if (!isValid) {
LOG.error(errorMsg);
LOG.warn("WriterSchema: " + writerSchema);
LOG.warn("Table latest schema: " + tableSchema);
if (isUpsert) {
throw new HoodieUpsertException(errorMsg, internalError);
} else {
throw new HoodieInsertException(errorMsg, internalError);
}
}
}
}

View File

@@ -52,6 +52,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
private static final String BASE_PATH_PROP = "hoodie.base.path";
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
private static final String AVRO_SCHEMA_VALIDATE = "hoodie.avro.schema.validate";
private static final String DEFAULT_AVRO_SCHEMA_VALIDATE = "false";
private static final String DEFAULT_PARALLELISM = "1500";
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
@@ -131,6 +133,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
props.setProperty(AVRO_SCHEMA, schemaStr);
}
public boolean getAvroSchemaValidate() {
return Boolean.parseBoolean(props.getProperty(AVRO_SCHEMA_VALIDATE));
}
public String getTableName() {
return props.getProperty(TABLE_NAME);
}
@@ -577,6 +583,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return this;
}
public Builder withAvroSchemaValidate(boolean enable) {
props.setProperty(AVRO_SCHEMA_VALIDATE, String.valueOf(enable));
return this;
}
public Builder forTable(String tableName) {
props.setProperty(TABLE_NAME, tableName);
return this;
@@ -721,6 +732,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
setDefaultOnCondition(props, !props.containsKey(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP),
FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP, DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
setDefaultOnCondition(props, !props.containsKey(AVRO_SCHEMA_VALIDATE), AVRO_SCHEMA_VALIDATE, DEFAULT_AVRO_SCHEMA_VALIDATE);
// Make sure the props is propagated
setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().fromProperties(props).build());

View File

@@ -191,8 +191,6 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
return Option.empty();
}
// TODO (NA) - Perform a writerSchema check of current input record with the last writerSchema on log file
// to make sure we don't append records with older (shorter) writerSchema than already appended
public void doAppend() {
while (recordItr.hasNext()) {
HoodieRecord record = recordItr.next();

View File

@@ -63,7 +63,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
this.partitionPath = partitionPath;
this.fileId = fileId;
this.originalSchema = new Schema.Parser().parse(config.getSchema());
this.writerSchema = createHoodieWriteSchema(originalSchema);
this.writerSchema = HoodieAvroUtils.createHoodieWriteSchema(originalSchema);
this.timer = new HoodieTimer().startTimer();
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
@@ -78,10 +78,6 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
return FSUtils.makeWriteToken(getPartitionId(), getStageId(), getAttemptId());
}
public static Schema createHoodieWriteSchema(Schema originalSchema) {
return HoodieAvroUtils.addMetadataFields(originalSchema);
}
public Path makeNewPath(String partitionPath) {
Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
try {