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,7 +25,7 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import org.apache.hudi.hive.util.SchemaUtil;
import org.apache.hudi.hive.util.HiveSchemaUtil;
import com.beust.jcommander.JCommander;
import org.apache.hadoop.conf.Configuration;
@@ -158,7 +158,7 @@ public class HiveSyncTool {
} else {
// Check if the table schema has evolved
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema(tableName);
SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields);
SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields);
if (!schemaDiff.isEmpty()) {
LOG.info("Schema difference found for " + tableName);
hoodieHiveClient.updateTableDefinition(tableName, schema);

View File

@@ -21,18 +21,14 @@ package org.apache.hudi.hive;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.StorageSchemes;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
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.table.TableSchemaResolver;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hudi.hive.util.SchemaUtil;
import org.apache.hudi.hive.util.HiveSchemaUtil;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -49,9 +45,6 @@ import org.apache.hadoop.hive.ql.session.SessionState;
import org.apache.hive.jdbc.HiveDriver;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.schema.MessageType;
import org.apache.thrift.TException;
@@ -74,7 +67,7 @@ public class HoodieHiveClient {
private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync";
// Make sure we have the hive JDBC driver in classpath
private static String driverName = HiveDriver.class.getName();
private static final String HIVE_ESCAPE_CHARACTER = SchemaUtil.HIVE_ESCAPE_CHARACTER;
private static final String HIVE_ESCAPE_CHARACTER = HiveSchemaUtil.HIVE_ESCAPE_CHARACTER;
static {
try {
@@ -250,7 +243,7 @@ public class HoodieHiveClient {
void updateTableDefinition(String tableName, MessageType newSchema) {
try {
String newSchemaStr = SchemaUtil.generateSchemaString(newSchema, syncConfig.partitionFields);
String newSchemaStr = HiveSchemaUtil.generateSchemaString(newSchema, syncConfig.partitionFields);
// Cascade clause should not be present for non-partitioned tables
String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : "";
StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append(HIVE_ESCAPE_CHARACTER)
@@ -268,7 +261,7 @@ public class HoodieHiveClient {
void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) {
try {
String createSQLQuery =
SchemaUtil.generateCreateDDL(tableName, storageSchema, syncConfig, inputFormatClass, outputFormatClass, serdeClass);
HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, syncConfig, inputFormatClass, outputFormatClass, serdeClass);
LOG.info("Creating table with " + createSQLQuery);
updateHiveSQL(createSQLQuery);
} catch (IOException e) {
@@ -340,122 +333,14 @@ public class HoodieHiveClient {
*
* @return Parquet schema for this table
*/
@SuppressWarnings("WeakerAccess")
public MessageType getDataSchema() {
try {
switch (tableType) {
case COPY_ON_WRITE:
// If this is COW, get the last commit and read the schema from a file written in the
// last commit
HoodieInstant lastCommit =
activeTimeline.lastInstant().orElseThrow(() -> new InvalidTableException(syncConfig.basePath));
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(activeTimeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class);
String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny()
.orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for commit "
+ lastCommit + ", could not get schema for table " + metaClient.getBasePath() + ", Metadata :"
+ commitMetadata));
return readSchemaFromBaseFile(new Path(filePath));
case MERGE_ON_READ:
// If this is MOR, depending on whether the latest commit is a delta commit or
// compaction commit
// Get a datafile written and get the schema from that file
Option<HoodieInstant> lastCompactionCommit =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
LOG.info("Found the last compaction commit as " + lastCompactionCommit);
Option<HoodieInstant> lastDeltaCommit;
if (lastCompactionCommit.isPresent()) {
lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants()
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE).lastInstant();
} else {
lastDeltaCommit =
metaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant();
}
LOG.info("Found the last delta commit " + lastDeltaCommit);
if (lastDeltaCommit.isPresent()) {
HoodieInstant lastDeltaInstant = lastDeltaCommit.get();
// read from the log file wrote
commitMetadata = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(lastDeltaInstant).get(),
HoodieCommitMetadata.class);
Pair<String, HoodieFileFormat> filePathWithFormat =
commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream()
.filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION)).findAny()
.map(f -> Pair.of(f, HoodieFileFormat.HOODIE_LOG)).orElseGet(() -> {
// No Log files in Delta-Commit. Check if there are any parquet files
return commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream()
.filter(s -> s.contains((metaClient.getTableConfig().getBaseFileFormat().getFileExtension())))
.findAny().map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() ->
new IllegalArgumentException("Could not find any data file written for commit "
+ lastDeltaInstant + ", could not get schema for table " + metaClient.getBasePath()
+ ", CommitMetadata :" + commitMetadata));
});
switch (filePathWithFormat.getRight()) {
case HOODIE_LOG:
return readSchemaFromLogFile(lastCompactionCommit, new Path(filePathWithFormat.getLeft()));
case PARQUET:
return readSchemaFromBaseFile(new Path(filePathWithFormat.getLeft()));
default:
throw new IllegalArgumentException("Unknown file format :" + filePathWithFormat.getRight()
+ " for file " + filePathWithFormat.getLeft());
}
} else {
return readSchemaFromLastCompaction(lastCompactionCommit);
}
default:
LOG.error("Unknown table type " + tableType);
throw new InvalidTableException(syncConfig.basePath);
}
} catch (IOException e) {
return new TableSchemaResolver(metaClient).getDataSchema();
} catch (Exception e) {
throw new HoodieHiveSyncException("Failed to read data schema", e);
}
}
/**
* Read schema from a data file from the last compaction commit done.
*/
private MessageType readSchemaFromLastCompaction(Option<HoodieInstant> lastCompactionCommitOpt) throws IOException {
HoodieInstant lastCompactionCommit = lastCompactionCommitOpt.orElseThrow(() -> new HoodieHiveSyncException(
"Could not read schema from last compaction, no compaction commits found on path " + syncConfig.basePath));
// Read from the compacted file wrote
HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata
.fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get(), HoodieCommitMetadata.class);
String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny()
.orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for compaction "
+ lastCompactionCommit + ", could not get schema for table " + metaClient.getBasePath()));
return readSchemaFromBaseFile(new Path(filePath));
}
/**
* Read the schema from the log file on path.
*/
private MessageType readSchemaFromLogFile(Option<HoodieInstant> lastCompactionCommitOpt, Path path)
throws IOException {
MessageType messageType = SchemaUtil.readSchemaFromLogFile(fs, path);
// Fall back to read the schema from last compaction
if (messageType == null) {
LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt);
return readSchemaFromLastCompaction(lastCompactionCommitOpt);
}
return messageType;
}
/**
* Read the parquet schema from a parquet File.
*/
private MessageType readSchemaFromBaseFile(Path parquetFilePath) throws IOException {
LOG.info("Reading schema from " + parquetFilePath);
if (!fs.exists(parquetFilePath)) {
throw new IllegalArgumentException(
"Failed to read schema from data file " + parquetFilePath + ". File does not exist.");
}
ParquetMetadata fileFooter =
ParquetFileReader.readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER);
return fileFooter.getFileMetaData().getSchema();
}
/**
* @return true if the configured table exists
*/

View File

@@ -18,20 +18,12 @@
package org.apache.hudi.hive.util;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HoodieHiveSyncException;
import org.apache.hudi.hive.SchemaDifference;
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;
import org.apache.parquet.schema.DecimalMetadata;
import org.apache.parquet.schema.GroupType;
import org.apache.parquet.schema.MessageType;
@@ -50,9 +42,9 @@ import java.util.Set;
/**
* Schema Utilities.
*/
public class SchemaUtil {
public class HiveSchemaUtil {
private static final Logger LOG = LogManager.getLogger(SchemaUtil.class);
private static final Logger LOG = LogManager.getLogger(HiveSchemaUtil.class);
public static final String HIVE_ESCAPE_CHARACTER = "`";
/**
@@ -424,25 +416,4 @@ public class SchemaUtil {
// Dont do that
return "String";
}
/**
* Read the schema from the log file on path.
*
* @return
*/
public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws IOException {
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
HoodieAvroDataBlock lastBlock = null;
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
if (block instanceof HoodieAvroDataBlock) {
lastBlock = (HoodieAvroDataBlock) block;
}
}
reader.close();
if (lastBlock != null) {
return new AvroSchemaConverter().convert(lastBlock.getSchema());
}
return null;
}
}

View File

@@ -22,7 +22,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import org.apache.hudi.hive.util.SchemaUtil;
import org.apache.hudi.hive.util.HiveSchemaUtil;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.parquet.schema.MessageType;
@@ -79,7 +79,7 @@ public class TestHiveSyncTool {
.optional(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list").named("int_list")
.named("ArrayOfInts");
String schemaString = SchemaUtil.generateSchemaString(schema);
String schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`int_list` ARRAY< int>", schemaString);
// A array of arrays
@@ -87,14 +87,14 @@ public class TestHiveSyncTool {
.as(OriginalType.LIST).repeatedGroup().required(PrimitiveType.PrimitiveTypeName.INT32).named("element")
.named("list").named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString);
// A list of integers
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeated(PrimitiveType.PrimitiveTypeName.INT32)
.named("element").named("int_list").named("ArrayOfInts");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`int_list` ARRAY< int>", schemaString);
// A list of structs with two fields
@@ -102,7 +102,7 @@ public class TestHiveSyncTool {
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").required(PrimitiveType.PrimitiveTypeName.INT32)
.named("num").named("element").named("tuple_list").named("ArrayOfTuples");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString);
// A list of structs with a single field
@@ -112,7 +112,7 @@ public class TestHiveSyncTool {
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array").named("one_tuple_list")
.named("ArrayOfOneTuples");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString);
// A list of structs with a single field
@@ -122,7 +122,7 @@ public class TestHiveSyncTool {
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list_tuple")
.named("one_tuple_list").named("ArrayOfOneTuples2");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString);
// A list of structs with a single field
@@ -132,7 +132,7 @@ public class TestHiveSyncTool {
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list").named("one_tuple_list")
.named("ArrayOfOneTuples3");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`one_tuple_list` ARRAY< binary>", schemaString);
// A list of maps
@@ -141,7 +141,7 @@ public class TestHiveSyncTool {
.as(OriginalType.UTF8).named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32).named("int_value")
.named("key_value").named("array").named("map_list").named("ArrayOfMaps");
schemaString = SchemaUtil.generateSchemaString(schema);
schemaString = HiveSchemaUtil.generateSchemaString(schema);
assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString);
}