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

@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
@@ -36,7 +37,6 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
import org.apache.hudi.hive.util.SchemaUtil;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.avro.Schema;
@@ -90,7 +90,7 @@ public class HoodieLogFileCommand implements CommandMarker {
for (String logFilePath : logFilePaths) {
FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath));
Schema writerSchema = new AvroSchemaConverter()
.convert(Objects.requireNonNull(SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePath))));
.convert(Objects.requireNonNull(TableSchemaResolver.readSchemaFromLogFile(fs, new Path(logFilePath))));
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
// read the avro blocks
@@ -179,7 +179,7 @@ public class HoodieLogFileCommand implements CommandMarker {
AvroSchemaConverter converter = new AvroSchemaConverter();
// get schema from last log file
Schema readerSchema =
converter.convert(Objects.requireNonNull(SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))));
converter.convert(Objects.requireNonNull(TableSchemaResolver.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))));
List<IndexedRecord> allRecords = new ArrayList<>();
@@ -202,7 +202,7 @@ public class HoodieLogFileCommand implements CommandMarker {
} else {
for (String logFile : logFilePaths) {
Schema writerSchema = new AvroSchemaConverter()
.convert(Objects.requireNonNull(SchemaUtil.readSchemaFromLogFile(client.getFs(), new Path(logFile))));
.convert(Objects.requireNonNull(TableSchemaResolver.readSchemaFromLogFile(client.getFs(), new Path(logFile))));
HoodieLogFormat.Reader reader =
HoodieLogFormat.newReader(fs, new HoodieLogFile(new Path(logFile)), writerSchema);
// read the avro blocks